Hi Philip,

Yeah, it's basically DNS resolution we're talking about, though there's
some additional subtlety there with the logic introduced by KIP-235 [1].
Essentially it should cover any scenario that causes a client constructor
to fail with the current logic but would not after this KIP is released.

We can generalize the Connect use case like this: a client application that
may connect to different Kafka clusters with a public-facing, easy-to-use
API for restarting failed tasks and automatic handling of retriable
exceptions. The ease with which failed tasks can be restarted is
significant because it reduces the cost of failing on non-retriable
exceptions and makes fail-fast behavior easier to work with. And, in cases
like this where we can't really know whether the error we're dealing with
is retriable or not, it's better IMO to continue to allow applications like
these to fail fast. I do agree that it'd be nice to get input from the
community, though.

I was toying with the idea of a NetworkException subclass too. It's a
simpler API, but it doesn't allow for preflight validation, which can be
useful in scenarios where submitting new configurations for client
applications is expensive in terms of time or resources. Then again, I
don't see why the two are mutually exclusive, and we might opt to use the
NetworkException subclass in this KIP and pursue an opt-in validation API
later on. Thoughts?

[1] -
https://cwiki.apache.org/confluence/display/KAFKA/KIP-235%3A+Add+DNS+alias+support+for+secured+connection

Cheers,

Chris

On Mon, Feb 27, 2023 at 7:06 PM Philip Nee <philip...@gmail.com> wrote:

> Hey Chris,
>
> Thanks again for the feedback!
>
>
> For the preflight DNS check (are we basically trying to resolve the DNS
> there?): Maybe it makes more sense to add it to the Config modules? I would
> like to hear what the community says as I'm not familiar with the Connect
> use case.
>
> A "slower failing" alternative - I wonder if it makes sense for us to
> extend the NetworkException so that clients can be smarter at handling
> these exceptions. Of course, it is still retriable and requires polling the
> consumer, but then we can distinguish the DNS resolution error from other
> network errors.
>
> Thanks!
> P
>
>
>
>
>
> On Mon, Feb 27, 2023 at 9:36 AM Chris Egerton <chr...@aiven.io.invalid>
> wrote:
>
> > Hi Philip,
> >
> > Yeah,  "DNS resolution should occur..." seems like a better fit. 👍
> >
> > One other question I have is whether we should expose some kind of public
> > API for performing preflight validation of the bootstrap URLs. If we
> change
> > the behavior of a client configured with a silly typo (e.g.,
> > "loclahost instead of localhost") from failing in the constructor to
> > failing with a retriable exception, this might lead some client
> > applications to handle that failure by, well, retrying. For reference,
> this
> > is exactly what we do in Kafka Connect right now; see [1] and [2]. IMO
> it'd
> > be nice to be able to opt into keeping the current behavior so that
> > projects like Connect could still do preflight checks of the
> > bootstrap.servers property for connectors before starting them, and
> report
> > any issues by failing fast instead of continuously writing warning/error
> > messages to their logs.
> >
> > I'm not sure about where this new API could go, but a few options might
> be:
> >
> > - Expose a public variant of the existing ClientUtils class
> > - Add static methods to the ConsumerConfig, ProducerConfig, and
> > AdminClientConfig classes
> > - Add those same static methods to the KafkaConsumer, KafkaProducer, and
> > KafkaAdminClient classes
> >
> > If this seems reasonable, we should probably also specify in the KIP that
> > Kafka Connect will leverage this preflight validation logic before
> > instantiating any Kafka clients for use by connectors or tasks, and
> > continue to fail fast if there are typos in the bootstrap.servers
> property,
> > or if temporary DNS resolution issues come up.
> >
> > [1] -
> >
> >
> https://github.com/apache/kafka/blob/5f9d01668cae64b2cacd7872d82964fa78862aaf/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java#L606
> > [2] -
> >
> >
> https://github.com/apache/kafka/blob/trunk/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java#L439
> >
> > Cheers,
> >
> > Chris
> >
> > On Fri, Feb 24, 2023 at 4:59 PM Philip Nee <philip...@gmail.com> wrote:
> >
> > > Hey Chris,
> > >
> > > Thanks for the quick response, and I apologize for the unclear wording
> > > there, I guess "DNS lookup" would be a more appropriate wording here.
> So
> > > what I meant there was, to delegate the DNS lookup in the constructor
> to
> > > the network client poll, and it will happen on the very first poll.  I
> > > guess the logic could look like this:
> > >
> > > - if the client has been bootstrapped, do nothing.
> > > - Otherwise, perform DNS lookup, and acquire the bootstrap server
> > address.
> > >
> > > Thanks for the comment there, I'll change up the wording.  Maybe revise
> > it
> > > as "DNS resolution should occur in the poll...." ?
> > >
> > > P
> > >
> > > On Fri, Feb 24, 2023 at 1:47 PM Chris Egerton <chr...@aiven.io.invalid
> >
> > > wrote:
> > >
> > > > Hi Philip,
> > > >
> > > > Thanks for the KIP!
> > > >
> > > > QQ: In the "Proposed Changes" section, the KIP states that
> > "Bootstrapping
> > > > should now occur in the poll method before attempting to update the
> > > > metadata. This includes resolving the addresses and bootstrapping the
> > > > metadata.". By "bootstrapping the metadata" do we mean actually
> > > contacting
> > > > the bootstrap servers, or just setting some internal state related to
> > the
> > > > current set of servers that can be contacted for metadata? I ask
> > because
> > > it
> > > > seems like the language here implies the former, but if that's the
> > case,
> > > > this is already happening in poll (or at least, the first invocation
> of
> > > > it), and if it's the latter, it's probably not necessary to mention
> in
> > > the
> > > > KIP since it doesn't really impact user-facing behavior. It also
> seems
> > > like
> > > > that detail might impact how intertwined this and KIP-899 are, though
> > the
> > > > similarity could still be superficial either way.
> > > >
> > > > Cheers,
> > > >
> > > > Chris
> > > >
> > > > On Thu, Feb 23, 2023 at 9:21 PM Philip Nee <philip...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Ismael,
> > > > >
> > > > > Thanks for the feedback! The proposal is not to retry automatically
> > but
> > > > > relies on the user polling the NetworkClient (basically,
> > consumer.poll)
> > > > to
> > > > > reattempt the bootstrap. If bootstrapping fails, a NetworkException
> > > > > (retriable) will be thrown.
> > > > >
> > > > > Thanks!
> > > > > P
> > > > >
> > > > >
> > > > >
> > > > > On Thu, Feb 23, 2023 at 1:34 PM Ismael Juma <ism...@juma.me.uk>
> > wrote:
> > > > >
> > > > > > Thanks for the KIP. Not sure if I missed it, but how long will we
> > > retry
> > > > > for
> > > > > > and when do we give up and propagate the failure to the user?
> > > > > >
> > > > > > Ismael
> > > > > >
> > > > > > On Thu, Feb 23, 2023 at 9:30 AM Philip Nee <philip...@gmail.com>
> > > > wrote:
> > > > > >
> > > > > > > Hi all!
> > > > > > >
> > > > > > > I want to start a discussion thread about how we can handle
> > client
> > > > > > > bootstrap failure due DNS lookup.  This requires a bit of
> > > behavioral
> > > > > > > change, so a KIP is proposed and attached to this email. Let me
> > > know
> > > > > what
> > > > > > > you think!
> > > > > > >
> > > > > > >
> > > > > > > *A small remark here*: *As the title of this KIP might sound
> > > > > > > familiar/similar to KIP-899, it is not the same.*
> > > > > > >
> > > > > > > *In Summary:* I want to propose a KIP to change the existing
> > > > bootstrap
> > > > > > > (upon instantiation) strategy because it is reasonable to allow
> > > > clients
> > > > > > to
> > > > > > > retry
> > > > > > >
> > > > > > > *KIP: *
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-909%3A+Allow+Clients+to+Rebootstrap+Upon+Failed+DNS+Resolution
> > > > > > >
> > > > > > > Thanks!
> > > > > > > Philip
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to