Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
Thanks Ivan! LGTM On Fri, Apr 12, 2024, 13:38 Ivan Yurchenko wrote: > Hi Chris and all, > > Thank you for your feedback. Your proposals seems good to me. I did these > changed to the KIP, please have a look at the change [1] > > Best, > Ivan > > [1] > https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=240881396&selectedPageVersions=14&selectedPageVersions=12 > > On Thu, Apr 11, 2024, at 10:49, Chris Egerton wrote: > > 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 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 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 ca
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
Hi Chris and all, Thank you for your feedback. Your proposals seems good to me. I did these changed to the KIP, please have a look at the change [1] Best, Ivan [1] https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=240881396&selectedPageVersions=14&selectedPageVersions=12 On Thu, Apr 11, 2024, at 10:49, Chris Egerton wrote: > 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 > 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 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 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
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 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 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
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
Hi Andrew and all, I did the mentioned change. As there are no more comments, I'm letting this sit for e.g. a week and then I will call for the vote. Best, Ivan On Tue, Apr 9, 2024, at 23:51, Andrew Schofield 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 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 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 > 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.
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 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 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
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 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 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
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 > >> 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
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 >> 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 >>> 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]).
RE: Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 > 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 > > 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?
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 > 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 >> 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 k
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 > 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 > > wrote: > > > > > Hi Iv
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 > 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 > > wrote: > > > > > Hi! > > >
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 > 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 > > 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 > > > > > > > > > > > >
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 > 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 > > > > > > >
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
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 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 > > >
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
Hi Philip and all, > if you don't use the client for a long time, why can't you just close the client and re-instantiate a new one when needed? I'm not familiar with the stream thread, so I don't know if that's possible. Yes, it's always possible to recreate the client (I think, it's the main workaround for this issue). Streams don't do this currently. If we do the fix on the Streams level (catch and recreate the client), it solves the problem there, but it remains unsolved fundamentally and it will be still necessary to do the same fix in other applications. > Another idea here is, would it make sense to expose a maybeUpdateMetadata() API to serve such a purpose? Could you please elaborate, what exactly this function should do? Best, Ivan On Tue, 24 Jan 2023 at 00:33, Philip Nee wrote: > Hey Ivan, > > Thanks for the KIP. Some questions for clarification: It seems like the > main problem is that if we don't poll frequently enough, the cluster > topology can change entirely before the metadata is refreshed and thus > causing staled clients. My question is: if you don't use the client for a > long time, why can't you just close the client and re-instantiate a new one > when needed? I'm not familiar with the stream thread, so I don't know if > that's possible. Another idea here is, would it make sense to expose a > maybeUpdateMetadata() API to serve such a purpose? > > Thanks, > P > > > On Wed, Jan 18, 2023 at 4:07 AM Ivan Yurchenko > 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 > > >
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
Hi Christo and all, > Currently a long-running client refreshes their metadata from a set of brokers obtained when first contacting the cluster. If they have been “away” for too long those brokers might have all changed and upon trying to refresh the metadata the client will fail because it cannot find an available broker. What you propose is that whenever such a situation is encountered the client should try to get the new set of brokers by communicating with the bootstrap-servers again. Yes, your understanding is correct. > To answer your question, in my opinion this behaviour should not be guarded by a configuration and should be the default once implemented. As a customer of Kafka, I cannot think of a situation where I would prefer my clients to give up if they have stale data without even trying to get the latest information. As far as I understand, the new behaviour will be entirely constrained to the client code which makes this change easier. This sounds reasonable. If we consider the current behavior as a bug, then the config is probably not needed for the "fix". It's a slight change of the client behavior and may be considered as incompatibility. However, I agree with you, it's hard to come up with a case where the current behavior is preferred over the proposed. I don't know what's the current approach to this in the community, would be great to hear some more opinions. > As a starting point can we confirm that this is indeed the current behaviour either by a reproducible manual test or by a branch with a failing unit/integration test? Yes, the reproduction is very easy and consistent. I reproduced this manually and I also have tests for both consumers and producers that fail without the fix [1] Best, Ivan. [1] https://github.com/ivanyu/kafka/blob/bd769d3eb76bf72f8aaf37553d6a1507dd7f8c55/core/src/test/scala/integration/kafka/api/RebootstrapTest.scala On Mon, 23 Jan 2023 at 12:26, Christo Lolov wrote: > Hello! > > Thank you for the KIP. I would like to summarise my understanding of the > problem in case I am wrong. > > Currently a long-running client refreshes their metadata from a set of > brokers obtained when first contacting the cluster. If they have been > “away” for too long those brokers might have all changed and upon trying to > refresh the metadata the client will fail because it cannot find an > available broker. What you propose is that whenever such a situation is > encountered the client should try to get the new set of brokers by > communicating with the bootstrap-servers again. > > If I have understood this correctly, then I agree with what is proposed as > a solution in this KIP. To answer your question, in my opinion this > behaviour should not be guarded by a configuration and should be the > default once implemented. As a customer of Kafka, I cannot think of a > situation where I would prefer my clients to give up if they have stale > data without even trying to get the latest information. As far as I > understand, the new behaviour will be entirely constrained to the client > code which makes this change easier. > > As a starting point can we confirm that this is indeed the current > behaviour either by a reproducible manual test or by a branch with a > failing unit/integration test? > > Best, > Christo > > > On 18 Jan 2023, at 12:07, Ivan Yurchenko > 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 >
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
Hey Ivan, Thanks for the KIP. Some questions for clarification: It seems like the main problem is that if we don't poll frequently enough, the cluster topology can change entirely before the metadata is refreshed and thus causing staled clients. My question is: if you don't use the client for a long time, why can't you just close the client and re-instantiate a new one when needed? I'm not familiar with the stream thread, so I don't know if that's possible. Another idea here is, would it make sense to expose a maybeUpdateMetadata() API to serve such a purpose? Thanks, P On Wed, Jan 18, 2023 at 4:07 AM Ivan Yurchenko 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 >
Re: [DISCUSS] KIP-899: Allow clients to rebootstrap
Hello! Thank you for the KIP. I would like to summarise my understanding of the problem in case I am wrong. Currently a long-running client refreshes their metadata from a set of brokers obtained when first contacting the cluster. If they have been “away” for too long those brokers might have all changed and upon trying to refresh the metadata the client will fail because it cannot find an available broker. What you propose is that whenever such a situation is encountered the client should try to get the new set of brokers by communicating with the bootstrap-servers again. If I have understood this correctly, then I agree with what is proposed as a solution in this KIP. To answer your question, in my opinion this behaviour should not be guarded by a configuration and should be the default once implemented. As a customer of Kafka, I cannot think of a situation where I would prefer my clients to give up if they have stale data without even trying to get the latest information. As far as I understand, the new behaviour will be entirely constrained to the client code which makes this change easier. As a starting point can we confirm that this is indeed the current behaviour either by a reproducible manual test or by a branch with a failing unit/integration test? Best, Christo > On 18 Jan 2023, at 12:07, Ivan Yurchenko 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
[DISCUSS] KIP-899: Allow clients to rebootstrap
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