Hi Ivan,

I agree with Andrew that we can save cluster ID checking for later. This
feature is opt-in and if necessary we can add a note to users about only
enabling it if they can be certain that the same cluster will always be
resolved by the bootstrap servers. This would apply regardless of whether
we did client ID checking anyways.

Thanks for exploring a variety of options and ironing out the details on
this KIP. I think this is acceptable as-is but have a couple of final
suggestions we might consider:

1. Although the definition of an unavailable broker is useful ("A broker is
unavailable when the client doesn't have an established connection with it
and cannot establish a connection (e.g. due to the reconnect backoff)"), I
think this is a little too restrictive. It's useful to note this as an
example of what we may consider an unavailable broker, but if we leave some
more wiggle room, it could save us the trouble of a follow-up KIP when
tweaking behavior in the future. For example, to reduce discovery time for
a migrated Kafka cluster, it could be nice to re-bootstrap after a
connection attempt has failed for every currently-known broker with no
successful attempts in between, instead of waiting for the reconnection
backoff interval to kick in. Again, I don't think this needs to happen with
the initial implementation of the KIP, I just want us to be able to explore
options like this in the future.

2. In a similar vein, I think we can leave more room in our definition of
re-bootstrapping. Instead of "During the rebootstrap process, the client
forgets the brokers it knows about and falls back on the bootstrap brokers
(i.e. provided by bootstrap.servers provided by the client configuration)
as if it had just been initialized.", we could say something like "During
the rebootstrap process, the client attempts to re-contact the bootstrap
servers (i.e. ...) that it contacted during initialization." This could be
useful if we want to add the bootstrap servers to the previously-known list
of brokers instead of completely discarding the previously-known set. This
too can be left out of the initial implementation and just give us a bit
more room for future options.

Cheers,

Chris

On Tue, Apr 9, 2024 at 11:51 AM Andrew Schofield <andrew_schofi...@live.com>
wrote:

> Hi Ivan,
> I think you have to go one way or the other with the cluster ID, so I
> think removing that from this KIP might
> be the best. I think there’s another KIP waiting to be written for
> ensuring consistency of clusters, but
> I think that wouldn’t conflict at all with this one.
>
> Thanks,
> Andrew
>
> > On 9 Apr 2024, at 19:11, Ivan Yurchenko <i...@ivanyu.me> wrote:
> >
> > Hi Andrew and all,
> >
> > I looked deeper into the code [1] and it seems the Metadata class is OK
> with cluster ID changing. So I'm thinking that the rebootstrapping
> shouldn't introduce a new failure mode here. And I should remove the
> mention of this cluster ID checks from the KIP.
> >
> > Best,
> > Ivan
> >
> > [1]
> https://github.com/apache/kafka/blob/ff90f78c700c582f9800013faad827c36b45ceb7/clients/src/main/java/org/apache/kafka/clients/Metadata.java#L355
> >
> > On Tue, Apr 9, 2024, at 09:28, Andrew Schofield wrote:
> >> Hi Ivan,
> >> Thanks for the KIP. I can see situations in which this would be
> helpful. I have one question.
> >>
> >> The KIP says the client checks the cluster ID when it re-bootstraps and
> that it will fail if the
> >> cluster ID doesn’t match the previously known one. How does it fail?
> Which exception does
> >> it throw and when?
> >>
> >> In a similar vein, now that we are checking cluster IDs, I wonder if it
> could be extended to
> >> cover all situations in which there are cluster ID mismatches, such as
> the bootstrap server
> >> list erroneously pointing at brokers from different clusters and the
> problem only being
> >> detectable later on.
> >>
> >> Thanks,
> >> Andrew
> >>
> >>> On 8 Apr 2024, at 18:24, Ivan Yurchenko <i...@ivanyu.me> wrote:
> >>>
> >>> Hello!
> >>>
> >>> I changed the KIP a bit, specifying that the certain benefit goes to
> consumers not participating in a group, but that other clients can benefit
> as well in certain situations.
> >>>
> >>> You can see the changes in the history [1]
> >>>
> >>> Thank you!
> >>>
> >>> Ivan
> >>>
> >>> [1]
> https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=240881396&originalVersion=10&revisedVersion=11
> >>>
> >>> On 2023/07/15 16:37:52 Ivan Yurchenko wrote:
> >>>> Hello!
> >>>>
> >>>> I've made several changes to the KIP based on the comments:
> >>>>
> >>>> 1. Reduced the scope to producer and consumer clients only.
> >>>> 2. Added more details to the description of the rebootstrap process.
> >>>> 3. Documented the role of low values of reconnect.backoff.max.ms in
> >>>> preventing rebootstrapping.
> >>>> 4. Some wording changes.
> >>>>
> >>>> You can see the changes in the history [1]
> >>>>
> >>>> I'm planning to put the KIP to a vote in some days if there are no new
> >>>> comments.
> >>>>
> >>>> Thank you!
> >>>>
> >>>> Ivan
> >>>>
> >>>> [1]
> >>>>
> https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=240881396&selectedPageVersions=9&selectedPageVersions=5
> >>>>
> >>>> On Tue, 30 May 2023 at 08:23, Ivan Yurchenko <iv...@gmail.com>
> >>>> wrote:
> >>>>
> >>>>> Hi Chris and all,
> >>>>>
> >>>>>> I believe the logic you've linked is only applicable for the
> producer and
> >>>>>> consumer clients; the admin client does something different (see
> [1]).
> >>>>>
> >>>>> I see, thank you for the pointer. It seems the admin client is fairly
> >>>>> different from the producer and consumer. Probably it makes sense to
> reduce
> >>>>> the scope of the KIP to the producer and consumer clients only.
> >>>>>
> >>>>>> it'd be nice to have a definition of when re-bootstrapping
> >>>>>> would occur that doesn't rely on internal implementation details.
> What
> >>>>>> user-visible phenomena can we identify that would lead to a
> >>>>>> re-bootstrapping?
> >>>>>
> >>>>> Let's put it this way: "Re-bootstrapping means that the client
> forgets
> >>>>> about nodes it knows about and falls back on the bootstrap nodes as
> if it
> >>>>> had just been initialized. Re-bootstrapping happens when, during a
> metadata
> >>>>> update (which may be scheduled by `metadata.max.age.ms` or caused by
> >>>>> certain error responses like NOT_LEADER_OR_FOLLOWER,
> REPLICA_NOT_AVAILABLE,
> >>>>> etc.), the client doesn't have a node with an established connection
> or
> >>>>> establishable connection."
> >>>>> Does this sound good?
> >>>>>
> >>>>>> I also believe that if someone has "
> >>>>>> reconnect.backoff.max.ms" set to a low-enough value,
> >>>>>> NetworkClient::leastLoadedNode may never return null. In that case,
> >>>>>> shouldn't we still attempt a re-bootstrap at some point (if the
> user has
> >>>>>> enabled this feature)?
> >>>>>
> >>>>> Yes, you're right. Particularly `canConnect` here [1] can always be
> >>>>> returning `true` if `reconnect.backoff.max.ms` is low enough.
> >>>>> It seems pretty difficult to find a good criteria when
> re-bootstrapping
> >>>>> should be forced in this case, so it'd be difficult to configure and
> reason
> >>>>> about. I think it's worth mentioning in the KIP and later in the
> >>>>> documentation, but we should not try to do anything special here.
> >>>>>
> >>>>>> Would it make sense to re-bootstrap only after "
> >>>>>> metadata.max.age.ms" has elapsed since the last metadata update,
> and
> >>>>> when
> >>>>>> at least one request has been made to contact each known server and
> been
> >>>>>> met with failure?
> >>>>>
> >>>>> The first condition is satisfied by the check in the beginning of
> >>>>> `maybeUpdate` [2].
> >>>>> It seems to me, the second one is also satisfied by
> `leastLoadedNode`.
> >>>>> Admittedly, it's more relaxed than you propose: it tracks
> unavailability of
> >>>>> nodes that was detected by all types of requests, not only by
> metadata
> >>>>> requests.
> >>>>> What do you think, would this be enough?
> >>>>>
> >>>>> [1]
> >>>>>
> https://github.com/apache/kafka/blob/c9a42c85e2c903329b3550181d230527e90e3646/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java#L698
> >>>>> [2]
> >>>>>
> https://github.com/apache/kafka/blob/c9a42c85e2c903329b3550181d230527e90e3646/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java#L1034-L1041
> >>>>>
> >>>>> Best,
> >>>>> Ivan
> >>>>>
> >>>>>
> >>>>> On Tue, 21 Feb 2023 at 20:07, Chris Egerton <ch...@aiven.io.invalid>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi Ivan,
> >>>>>>
> >>>>>> I believe the logic you've linked is only applicable for the
> producer and
> >>>>>> consumer clients; the admin client does something different (see
> [1]).
> >>>>>>
> >>>>>> Either way, it'd be nice to have a definition of when
> re-bootstrapping
> >>>>>> would occur that doesn't rely on internal implementation details.
> What
> >>>>>> user-visible phenomena can we identify that would lead to a
> >>>>>> re-bootstrapping? I also believe that if someone has "
> >>>>>> reconnect.backoff.max.ms" set to a low-enough value,
> >>>>>> NetworkClient::leastLoadedNode may never return null. In that case,
> >>>>>> shouldn't we still attempt a re-bootstrap at some point (if the
> user has
> >>>>>> enabled this feature)? Would it make sense to re-bootstrap only
> after "
> >>>>>> metadata.max.age.ms" has elapsed since the last metadata update,
> and when
> >>>>>> at least one request has been made to contact each known server and
> been
> >>>>>> met with failure?
> >>>>>>
> >>>>>> [1] -
> >>>>>>
> >>>>>>
> https://github.com/apache/kafka/blob/c9a42c85e2c903329b3550181d230527e90e3646/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminMetadataManager.java#L100
> >>>>>>
> >>>>>> Cheers,
> >>>>>>
> >>>>>> Chris
> >>>>>>
> >>>>>> On Sun, Feb 19, 2023 at 3:39 PM Ivan Yurchenko <iv...@gmail.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> 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 <ch...@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