KAFKA-4024: Fix metadata update to not apply backoff until it experiences failure#1707
KAFKA-4024: Fix metadata update to not apply backoff until it experiences failure#1707kawamuray wants to merge 18 commits intoapache:trunkfrom
Conversation
fc02b23 to
02cbc6d
Compare
|
Think I would follow up more unit tests. |
ee2eceb to
2e44c9d
Compare
|
@ijuma Can you review this please? |
There was a problem hiding this comment.
Do we need a comment? It seems obvious from the code, no?
There was a problem hiding this comment.
Well, when I first read this method it was bit hard to understand which value would be adopted, so I thought small comments would help new readers but I'm fine to remove these comments if you think it's not that helpful.
|
Thanks for the PR, LGTM (apart from some minor comments). The documentation for "The amount of time to wait before attempting to retry a failed request to a given topic partition. This avoids repeatedly sending requests in a tight loop under some failure scenarios." @hachikuji, can you double-check that this change is fine? |
|
One more thing, do we want to make the first The producer does: this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds());The consumer does: this.metadata.update(Cluster.bootstrap(addresses), 0); |
Yeah actually this is how this PR first look like since I thought this is the problem that happens only on the first send. |
5aa67ce to
8d59a9e
Compare
|
@kawamuray There are a bunch of cases in the clients where we can refetch metadata in a tight loop (e.g. if a partition leader is not available). The metadata fetches are successful, but we are waiting for a condition inside the response to become true. Previously the backoff protected us in this case, but we seem to be losing that, right? Is this not a concern? |
|
@hachikuji That's a good point. If we want to continue using it in those cases, we probably need to update the config documentation. |
|
@ijuma Yeah, agreed. Seems like the problem is that the definition of "failed request" might not be too clear in the documentation. I think what we really mean is any metadata fetch containing errors, which covers topics not existing, partition leaders not available, etc. These cases are considered "successful" updates from the perspective of the
Thoughts? |
|
@hachikuji Thanks for your comment. |
|
@kawamuray Hmm.. interesting point. We do get some protection from the reconnect backoff in some cases, but I'm still a little uneasy in general about changing the backoff behavior. The metadata requests can be harmful, especially in some cases for the consumer where we have to fetch metadata for all topics (i.e. regex subscriptions). We've seen cases where a bunch of clients fetching metadata in a tight loop have brought the cluster to its knees. What do you think about my suggestions above? We essentially keep the current behavior, but we bypass the backoff if the set of topics has changed since the last refresh. |
Hmm, I think this approach still doesn't work for following case?
|
|
@kawamuray I think a longer delay to respond to failures is basically the price you have to pay to avoid overwhelming the servers. It takes a little time on the server side to detect the failure and migrate partitions anyway, so I don't think immediately sending a metadata update after the failure would have much value, though I could be wrong. I'd prefer for now to leave the current backoff behavior in tact except when we know something in the request has changed. Perhaps it would make sense to consider in a follow-up issue whether something like exponential backoff would be a more ideal way to address this problem in practice. |
I think this isn't true for some cases. For example, when we perform rolling restart of brokers, each broker will shutdown gracefully so the controller can detect shutdown broker much faster than waiting zk session timeout.
However I agree with this. At least in case of broker failure, the first immediate metadata request likely results unchanged metadata, which results producer to wait Actually I'm bit confusing now. The reason I lengthened Then the first case(the first producer.send() blocked inappropriately) is the only problem as long as we keep Anyway, I strongly agree with that it's much better to have exponential backoff or just an idea but something like "listen to metadata update" mechanism which sounds like not that hard to implement as we already have purgatory implementation in server side, in order to minimize MTTR. WDYT? |
|
@hachikuji
I read through your comments again and realized that "set of topics" is producer.send(new ProducerRecord("topic-A", ...));
producer.send(new ProducerRecord("topic-B", ...)); // Maybe blocked refreshBackoffMsOf course it prevents the first problem as well. |
|
@kawamuray Yeah, sounds good, sorry if my suggestion wasn't clear at first (I've been looking at this code a little too long so I just assume everyone else has as well). Also good to know that we won't exhaust the retry limit for produce requests when dealing with partition leader failover. |
8d59a9e to
64ec2ec
Compare
|
@hachikuji Updated PR based on your suggestion. I modified the test code like this: https://gist.github.com/kawamuray/fdfcfec85f179417c3a2d4866f055df5 and ran against 9c24e2d then the result was look like below: This reuslt has 2 problems:
I digged into DefaultMetadataUpdater and found that timeToNextReconnectAttempt is causing this. The cause of problem 1 is that DefaultMetadataUpdater updates After appling 64ec2ec, the experiment result look like this: |
|
@hachikuji ping. |
|
@kawamuray Thanks for the reminder. I'll look at this in the morning. |
hachikuji
left a comment
There was a problem hiding this comment.
@kawamuray Left a couple comments. It's a nice find that the current client does a bunch of unnecessary connection attempts when fetching metadata initially.
There was a problem hiding this comment.
Can you split this into several separate test cases? It makes the tests more useful (since they are more focused) and easier to maintain. At a minimum, you can have different cases for Metadata.setTopics, Metadata.add, and Metadata. needMetadataForAllTopics.
There was a problem hiding this comment.
Maybe we should have a isDisconnected method in ClusterConnectionStates?
There was a problem hiding this comment.
Wouldn't it make more sense to check lastConnectAttemptNodeId in maybeUpdate(long now) before finding a new node with leastLoadedNode?
There was a problem hiding this comment.
hmm, is it possible?
guess it doesn't work in following case?
- Have two nodes. Node-A(disconnected), Node-B(connected, in-flight-requests is 1)
leastLoadedNode()returns Node-A which isn't connected yet. Initiate connection to Node-A.- On the next
poll(), a response from Node-B received so the in-flight-requests is now 0. - On next
maybeUpdate()leastLoadedNode()returns Node-B.
There was a problem hiding this comment.
This is a slight change in behavior, right? Previously if we cannot find any node to connect to, we'll block in poll() for the retry backoff. Now it seems we'll end up calling poll() in a tight loop until we can find a node to connect to. Maybe the logic in maybeUpdate needs to be something like this:
- Check if there is a connection in progress. If so, return the reconnect backoff as the timeout.
- If there is no connection in progress, call
leastLoadedNodeto find some node to connect to. If this returns null, return retry backoff as is currently done. - Next, if we have a node to connect to, then we initiate the connect (if disconnected) and set
lastConnectAttemptNodeId. - Finally, if already connected, send the metadata request.
Would that keep the current backoff behavior while avoiding the unnecessary connection attempts?
There was a problem hiding this comment.
You're right.
Even though maybeUpdate() returns the maximum time to wait as a backoff, network event(i.e, connection establishment, receive response) can trigger immediate progress to the next step.
I basically agree with your four steps except one thing about 2.
So whenever leastLoadedNode returns null, that means all nodes are currently backing off right? In case, we should return the minimum remaining reconnect backoff time of all nodes I guess. I will add a followup commit based on this. let's make further discussion on updated patch if necessary.
Thanks for nice catch.
…her than checking if the last connect attempt failed
There was a problem hiding this comment.
Hmm, after I added fdb0984, I found that a test start failing.
org.apache.kafka.clients.consumer.KafkaConsumerTest > testChangingRegexSubscription FAILED
java.lang.IllegalStateException: Next in line response did not match expected request
at org.apache.kafka.clients.MockClient.send(MockClient.java:147)
at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.trySend(ConsumerNetworkClient.java:392)
at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:223)
at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180)
at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:308)
at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:277)
at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:259)
at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1012)
at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:978)
at org.apache.kafka.clients.consumer.KafkaConsumerTest.testChangingRegexSubscription(KafkaConsumerTest.java:623)
This is because the current regexp-based subscription relies on Metadata.Listener to update the list of topics that it subscribing on: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L176
So this line still isn't removable even though we apply fdb0984.
However it might still make sense to keep the change as it reduced few LOC in other situations and keep calling requestUpdate() explicitly for the cases like this.
I'm on the line to revert fdb0984 as I think it is making the responsibility of calling requestUpdate() among caller and Metadata but
Just added 944f9f8 but still wanna hear your opinion @hachikuji .
There was a problem hiding this comment.
Good catch! Yeah, I forgot that we should always request a new metadata update when changing the regex subscription, even though it doesn't change anything about the topics we are fetching metadata for.
…couldn't be removed
2e93b75 to
944f9f8
Compare
|
LGTM. Thanks for the patch! I'm going to run some system tests prior to merging to make sure we didn't miss anything obvious, but hopefully we can wrap this up shortly. |
|
Quick update, there were some system test failures: http://testing.confluent.io/confluent-kafka-branch-builder-system-test-results/?prefix=2016-10-31--001.1477945677--hachikuji--KAFKA-4024--944f9f8/. I see a lot of messages like this from the consumer: Haven't had time yet to dig into it to see if it is related to any changes in this patch. |
|
Doh! super good catch @hachikuji . I ran the system test locally and it reproduced exactly. I turned on TRACE logging of VerifiableProducer and found that it stops requesting update of metadata after it received following response from the broker, which is incomplete as the leader of partition 0 is missing, then it never continue producing accumulated records even though broker failovered successfully: This is because we made this change: fdb0984#diff-499d1af91cfaf42966a8e2e485a1b4b9L180
|
|
@kawamuray Good find and thanks for investigating. |
|
Merged to trunk. Thanks again for the very delicate patch. We'll have to keep an eye on the nightly builds to make sure we didn't miss anything. 😅 |
|
We had a long run :D Thanks so much for your insightful review @hachikuji! |
|
Oh, forgot to mention. Prior to merging, I added a little extra explanation to one of the comments in the code. Hopefully it's reasonable. |
Issue: https://issues.apache.org/jira/browse/KAFKA-4024
Fixes a bug that inappropriately applies backoff as interval between metadata updates even though the current one is outdated.