Hi Chris,

Thank you for your question. As a part of various lifecycle phases
(including node disconnect), NetworkClient can request metadata update
eagerly (the `Metadata.requestUpdate` method), which results in
`MetadataUpdater.maybeUpdate` being called during next poll. Inside, it has
a way to find a known node it can connect to for the fresh metadata. If no
such node is found, it backs off. (Code [1]). I'm thinking of piggybacking
on this logic and injecting the rebootstrap attempt before the backoff.

As of the second part of you question: the re-bootstrapping means replacing
the node addresses in the client with the original bootstrap addresses, so
if the first bootstrap attempt fails, the client will continue using the
bootstrap addresses until success -- pretty much as if it were recreated
from scratch.

Best,
Ivan

[1]
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java#L1045-L1049

On Thu, 16 Feb 2023 at 17:18, Chris Egerton <chr...@aiven.io.invalid> wrote:

> Hi Ivan,
>
> I'm not very familiar with the clients side of things but the proposal
> seems reasonable.
>
> I like the flexibility of the "metadata.recovery.strategy" property as a
> string instead of, e.g., a "rebootstrap.enabled" boolean. We may want to
> adapt a different approach in the future, like the background thread
> mentioned in the rejected alternatives section.
>
> I also like handling this via configuration property instead of adding a
> Java-level API or suggesting that users re-instantiate their clients since
> we may want to enable this new behavior by default in the future, and it
> also reduces the level of effort required for users to benefit from this
> improvement.
>
> One question I have--that may have an obvious answer to anyone more
> familiar with client internals--is under which conditions we will determine
> a rebootstrap is appropriate. Taking the admin client as an example, the "
> default.api.timeout.ms" property gives us a limit on the time an operation
> will be allowed to take before it completes or fails (with optional
> per-request overrides in the various *Options classes), and the "
> request.timeout.ms" property gives us a limit on the time each request
> issued for that operation will be allowed to take before it completes, is
> retried, or causes the operation to fail (if no more retries can be
> performed). If all of the known servers (i.e., bootstrap servers for the
> first operation, or discovered brokers if bootstrapping has already been
> completed) are unavailable, the admin client will keep (re)trying to fetch
> metadata until the API timeout is exhausted, issuing multiple requests to
> the same server if necessary. When would a re-bootstrapping occur here?
> Ideally we could find some approach that minimizes false positives (where a
> re-bootstrapping is performed even though the current set of known brokers
> is only temporarily unavailable, as opposed to permanently moved). Of
> course, given the opt-in nature of the re-bootstrapping feature, we can
> always shoot for "good enough" on that front, but, it'd be nice to
> understand some of the potential pitfalls of enabling it.
>
> Following up on the above, would we cache the need to perform a
> re-bootstrap across separate operations? For example, if I try to describe
> a cluster, then a re-bootstrapping takes place and fails, and then I try to
> describe the cluster a second time. With that second attempt, would we
> immediately resort to the bootstrap servers for any initial metadata
> updates, or would we still try to go through the last-known set of brokers
> first?
>
> Cheers,
>
> Chris
>
> On Mon, Feb 6, 2023 at 4:32 AM Ivan Yurchenko <ivan0yurche...@gmail.com>
> wrote:
>
> > Hi!
> >
> > There seems to be not much more discussion going, so I'm planning to
> start
> > the vote in a couple of days.
> >
> > Thanks,
> >
> > Ivan
> >
> > On Wed, 18 Jan 2023 at 12:06, Ivan Yurchenko <ivan0yurche...@gmail.com>
> > wrote:
> >
> > > Hello!
> > > I would like to start the discussion thread on KIP-899: Allow clients
> to
> > > rebootstrap.
> > > This KIP proposes to allow Kafka clients to repeat the bootstrap
> process
> > > when fetching metadata if none of the known nodes are available.
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-899%3A+Allow+clients+to+rebootstrap
> > >
> > > A question right away: should we eventually change the default behavior
> > or
> > > it can remain configurable "forever"? The latter is proposed in the
> KIP.
> > >
> > > Thank you!
> > >
> > > Ivan
> > >
> > >
> > >
> >
>

Reply via email to