KAFKA-5697: prevent poll() from blocking forever#4855
KAFKA-5697: prevent poll() from blocking forever#4855hachikuji merged 51 commits intoapache:trunkfrom vvcephei:kafka-5697-stream-thread-shutdown
Conversation
|
@guozhangwang I think this change is a good solution to the problem you reported in https://issues.apache.org/jira/browse/KAFKA-5697. WDYT? I've tried tomake the change in a 'private' fashion so as not to Who else should review a change like this? |
|
Thanks for the patch. I knew we should be pretty close to being able to do this by now after some of the recent work. Note, however, that The main thing to be careful of is probably the case you mentioned where users have depended on |
|
I'll leave to @hachikuji and @mjsax to review and unblock you. |
|
Hey @hachikuji, Thanks for taking a look! I'll add a timeout to I was unaware that folks would use In that light, I do think I should go ahead and do a KIP, since with this change there would be no way to wait for an assignment without polling. That will also be a good way to seek feedback on whether or not we need the drain-previously-fetched use case, since that will also no longer be possible. And of course, users may need to increase their timeout value since the metadata updates will now count against it. If it's not too distracting to you, I'll keep this PR open to sketch out the new method and to update the tests and work though any failures while I write the KIP. |
|
@vvcephei Yeah, that sounds good to me. I'm not sure how common that usage is--it's definitely convenient in testing--but it's probably a good idea to get some more visibility for the change to be on the safe side. Thanks for picking this up, by the way! |
|
Just to record my current state on this... I decided a good way to approach this was to lay the groundwork by introducing timeouts to every call in the I found in testing that altering the semantics of Of course, the new timeouts are of no value unless they are exposed in some way, and that's what I'm going to work on tomorrow. Once I have poked around enough to have a decent proposal, then I'll write the promised KIP. |
|
rebased |
|
|
It's been a while since I've recorded status on this. I have a theory that Thus, if we need to refresh metadata, and let's say that metadata operation would take 2s and then we have some records to grab, and that would take 3s, then all these scenarios would work as expected: completely async:completely sync:sync, but not long enough to complete the full operation:In other words, we make poll completely async but leave in the option to block for responses, since that makes some use cases (especially tests and POCs simpler). Even with some timeout bigger than 0 but less than infinity, it's ok to block as long as we can and then return an empty list if we didn't finish, whether the thing we didn't finish was a metadata update or a fetch request. This is ok because callers are supposed to poll I needed to validate that metadata updates can actually function in an async manner, so (after I added a timeout to the metadata update) I set the metadata update timeout to 0 and put it in a tight loop that would continue until it completed. This is equivalent to the current "block until metadata update completes", but it'll only work if async metadata updates is implemented correctly. |
|
The tests passed on JDK7 and JDK10. The failure on JDK8 looks like an unrelated failure out of Zookeeper: Retest this, please. |
|
Now that I've validated the approach, I've added the new methods I'm proposing and deprecating the old poll. |
|
One remaining task before this is mergable is to update at least some tests to use either the new |
|
@hachikuji @ConcurrencyPractitioner , I've updated this PR with the latest state from KIP-266, as a more concrete example of the proposal. WDYT? Two things came up for me going through this...
Alternatives: we could coerce to MAX_VALUE ourselves, or we could use a loop to actually wait for any duration specified. I'm not sure either of those points are super relevant to the KIP discussion (unless you think so), so I'll leave it to you to bring them up if you're concerned. I'll be out for a week, so I won't be available to reply to any comments until then. Note that the tests for this PR won't pass until the CIT agents are updated to java 8 (but you should be able to build it yourself if desired). |
|
Hi @vvcephei, Thanks for setting up the basic methods in the interfaces, but you could just stick with implement poll. |
There was a problem hiding this comment.
Can you move this to after line 95 ?
There was a problem hiding this comment.
24.86 days is not a month :-)
There was a problem hiding this comment.
Truth! But I wanted to convey the ballpark to help readers and maintainers understand that unpredicated calls will block for quite a long time, but not "effectively forever", which is the mental mistake I made when I read "max long milliseconds". So now, I have this mental scale: "max long" seconds is effectively forever, with milliseconds it's a month-ish, and with nanoseconds it's about a week.
There was a problem hiding this comment.
Shouldn't the time spent in this loop be counted toward remaining ?
There was a problem hiding this comment.
Good eye. But this is the "old" method, whose behavior I'm preserving. So this is a re-expression of the existing behavior, that metadata update will block arbitrarily long and not count against the timeout, followed by the fetch, to which the timeout applies.
|
@vvcephei KAFKA-5697 has been fixed via GitHub Pull Request #4930, so I'm wondering if it is more for KIP-266 itself? |
|
retest this please |
|
@ConcurrencyPractitioner Sure thing; I'll pull the extra stuff back out. it wasn't as trivial as I thought it would be, evidenced by the fact that the tests broke, so I agree that poll alone is a big enough piece for one PR. |
|
@tedyu Thanks for the reivew! The test failures were related. Something in the "extra methods" I popped in caused the test failures, but I'm about to take them back out, so we should be green again. |
|
Ok, bringing this PR back to life. I've pulled out the "extra" methods I'd added. I expect the tests to pass. Next up, I need to write a few tests for the deprecated method and migrate the rest of the code base and tests away from it. @hachikuji should I just migrate the |
|
@vvcephei thanks for the updated PR, please ping me and @hachikuji whenever it is ready to be reviewed. Please do the following to help reviewers better understand the PR diffs:
|
Several tests depended on being able to send a timeout of 0, but still have the coordinator poll take non-zero time to do its work. I updated them to send a long enough timeout for the coordinator to to the required work.
hachikuji
left a comment
There was a problem hiding this comment.
Thanks for the patch. Left a few comments.
| // after the long poll, we should check whether the group needs to rebalance | ||
| // prior to returning data so that the group can stabilize faster | ||
| if (coordinator.needRejoin()) | ||
| if (coordinator.needRejoin()) { |
There was a problem hiding this comment.
nit: avoiding the braces has been the convention in kafka. I was never too fond of the convention and I'm not strict about enforcing it for new code, but it's a little annoying and distracting to change the old code.
There was a problem hiding this comment.
ok, sorry; it happened in a few places when I applied auto-formatting to the methods I was touching. I'll make a pass through and revert formatting changes.
| return this.interceptors.onConsume(new ConsumerRecords<>(records)); | ||
| } | ||
| final long fetchEnd = time.milliseconds(); | ||
| remainingMs = Math.max(0L, remainingMs - (fetchEnd - fetchStart)); |
There was a problem hiding this comment.
We don't need super precise timing control for any of these APIs, but the advantage of the previous approach is that the timeout is effectively a deadline and you won't accrue rounding errors while updating the time remaining. I'm not sure what the reason to change the logic was.
There was a problem hiding this comment.
Yeah... This is a result of double-refactoring. I got away from the "elapsed" approach when I introduced the "remainingTimeAtLeastZero" method, and then I made a recent change to avoid calling system time so much. I would have gone back to the original time-tracking approach, but I'd forgotten about it.
You are right, tracking elapsed time doesn't rack up rounding errors. I'll add another patch momentarily using this approach throughout my diff instead of "remainingTime".
|
|
||
| long startMs = time.milliseconds(); | ||
| coordinator.poll(startMs, timeout); | ||
| return updateFetchPositions(remainingMsAtLeastZero(startMs, timeout)); |
There was a problem hiding this comment.
Since Coordinator.poll() can block, we probably have to update startMs here.
There was a problem hiding this comment.
That's why we're calling remainingMsAtLeastZero for a new timeout here. Although I might be about to change this to use elapsed-time instead.
| // We do not want to be stuck blocking in poll if we are missing some positions | ||
| // since the offset lookup may be backing off after a failure | ||
| if (!hasAllFetchPositions && pollTimeout > retryBackoffMs) | ||
| if (!subscriptions.hasAllFetchPositions() && pollTimeout > retryBackoffMs) { |
There was a problem hiding this comment.
The previous code tried to avoid unnecessary passes over the assigned partitions since the set can get fairly large. I'm not sure how beneficial this optimization was. Maybe it's fine to lose it. Nevertheless, since poll can be called quite tightly, it would be nice to avoid redundant work.
There was a problem hiding this comment.
hmm. I'm not sure what a good way to do this is... hasAllFetchPositions was the result of updateAllFetchPositions before (which changed because the result now indicates whether it timed out or not). But updateAllFetchPositions is not longer called in this scope, so we can't consider keeping its result around.
I could cache the "hasAllFetchPositions" fact on a field in KafkaConsumer, but it seems a little risky. I think it would be fine... WDYT?
| */ | ||
| protected synchronized boolean ensureCoordinatorReady(long startTimeMs, long timeoutMs) { | ||
| long remainingMs = timeoutMs; | ||
| public synchronized boolean ensureCoordinatorReady(final long timeoutMs) { |
There was a problem hiding this comment.
It was actually because of the point mentioned before about avoiding unnecessary system calls. It's just kind of annoying when you have to pass it down deep into the stack, so we may not have followed the pattern consistently.
Also, why change this to public?
| } | ||
|
|
||
| private Set<TopicPartition> pendingCommittedOffsetRequest = null; | ||
| private RequestFuture<Map<TopicPartition, OffsetAndMetadata>> pendingCommittedOffsetResponse = null; |
There was a problem hiding this comment.
Can we move these to the top with the rest of the fields? Maybe we can choose clearer names?
There was a problem hiding this comment.
yep. I'll also wrap them in an inner class, since we're going to add generation too.
|
|
||
| private Set<TopicPartition> pendingCommittedOffsetRequest = null; | ||
| private RequestFuture<Map<TopicPartition, OffsetAndMetadata>> pendingCommittedOffsetResponse = null; | ||
|
|
There was a problem hiding this comment.
I think the basic idea makes sense and is what I was expecting. It might not feel too elegant, but I think a simple approach is best initially. An interesting point to consider is what would happen if an offset fetch is in-flight while a rebalance is in progress. When it returns, the offsets may be stale. I am wondering if it makes sense to fence the response using the group generation. In other words, we record the generation at the time of the request and then verify when we receive the response that it matches the current group generation.
| */ | ||
| public void ensureFreshMetadata() { | ||
| if (this.metadata.updateRequested() || this.metadata.timeToNextUpdate(time.milliseconds()) == 0) | ||
| if (this.metadata.updateRequested() || this.metadata.timeToNextUpdate(time.milliseconds()) == 0) { |
There was a problem hiding this comment.
Could we replace this with ensureFreshMetadata(Long.MAX_VALUE)?
|
|
||
| boolean ensureFreshMetadata(final long timeout) { | ||
| if (timeout < 0) { | ||
| return false; |
There was a problem hiding this comment.
Could we just raise IllegalArgumentException as we do in most other cases?
| import static org.junit.Assert.fail; | ||
|
|
||
| @SuppressWarnings({"deprecation", "SameParameterValue"}) | ||
| public class DeprecatedMethodsKafkaConsumerTest { |
There was a problem hiding this comment.
There has to be a better way to do this. For example, maybe we can extend KafkaConsumerTest?
|
Allrighty, @hachikuji, Thanks so much for the thorough review. I think I've addressed all your concerns. Regarding the timeout-vs-deadline discussion. I've refactored all my timeout handling in terms of elapsed time and minimized the system calls. I think this is good enough for now. I would agree with switching over to deadlines in the future, but it feels too big to lump in with this work. Thanks, |
hachikuji
left a comment
There was a problem hiding this comment.
Thanks for the updates. Just a few more comments.
|
|
||
| // NOTE: the use of cachedSubscriptionHashAllFetchPositions means we MUST call | ||
| // updateAssignmentMetadataIfNeeded before this method. | ||
| if (!cachedSubscriptionHashAllFetchPositions && pollTimeout > retryBackoffMs) { |
There was a problem hiding this comment.
We might also consider moving this into SubscriptionState? We can invalidate the value whenever the assignment changes or an offset is reset. That would also give us a shortcut for missingFetchPositions().
There was a problem hiding this comment.
I think we'd better defer this for later.
|
|
||
| private synchronized boolean rejoinIncomplete() { | ||
| return joinFuture != null; | ||
| protected synchronized boolean rejoinNeededOrPending() { |
There was a problem hiding this comment.
We should probably update the javadoc
| } | ||
|
|
||
| private boolean sameRequest(final Set<TopicPartition> currentRequest, final Generation currentGeneration) { | ||
| return currentGeneration.generationId == generation.generationId |
There was a problem hiding this comment.
Maybe we can implement equals for Generation?
There was a problem hiding this comment.
I considered it, but wanted to make sure this was in fact the comparison we should be doing here. I guess it is.
|
|
||
| // expose for tests | ||
| @Override | ||
| protected synchronized boolean ensureCoordinatorReady(final long timeoutMs) { |
There was a problem hiding this comment.
Do we need this? Seems like the same signature as the parent.
There was a problem hiding this comment.
It's weird, but doing this makes the method accessible in the WorkerCoordinatorTest, which it isn't otherwise because the test is in a different package from AbstractCoordinator. WorkerCoordinator can see the superclass's method because it is a subclass.
| assertTrue(s"Found unexpected threads during $context, allThreads=$threads", noUnexpected) | ||
| assertTrue( | ||
| s"Found unexpected threads during $context, allThreads=$threads, " + | ||
| s"unexpected=${threads.filterNot(t => unexpectedThreadNames.forall(s => !t.contains(s)))}", |
|
Ok, @hachikuji , I think this PR is gtg now, supposing the tests pass. |
| // update metadata (if needed) and keep track of the metadata used for assignment so that | ||
| // we can check after rebalance completion whether anything has changed | ||
| client.ensureFreshMetadata(); | ||
| if (!client.ensureFreshMetadata(Long.MAX_VALUE)) throw new TimeoutException(); |
There was a problem hiding this comment.
I guess it's tough to avoid blocking here. All of this would be considerably easier if we moved the rebalance to the background thread. Another improvement for another time.
hachikuji
left a comment
There was a problem hiding this comment.
LGTM. Thanks for the patch
| private PendingCommittedOffsetRequest pendingCommittedOffsetRequest = null; | ||
|
|
||
| private static class PendingCommittedOffsetRequest { | ||
| private final Set<TopicPartition> request; |
There was a problem hiding this comment.
The names here are a bit misleading, maybe we can fix them in #5087? cc @hachikuji . I'm thinking requestedPartitions and requestedGeneration?
| return true; | ||
| } | ||
|
|
||
| private long remainingTimeAtLeastZero(final long timeoutMs, final long elapsed) { |
There was a problem hiding this comment.
We have three remainingTimeAtLeastZero functions, in AbstractCoordinator, ConsumerCoordinator and KafkaConsumer. Is it intentional? If not we could leave just one to avoid unintentional code divergence in the future. cc @vvcephei
Add the new stricter-timeout version of `poll` proposed in KIP-266.
The pre-existing variant `poll(long timeout)` would block indefinitely for metadata
updates if they were needed, then it would issue a fetch and poll for `timeout` ms
for new records. The initial indefinite metadata block caused applications to become
stuck when the brokers became unavailable. The existence of the timeout parameter
made the indefinite block especially unintuitive.
This PR adds `poll(Duration timeout)` with the semantics:
1. iff a metadata update is needed:
1. send (asynchronous) metadata requests
2. poll for metadata responses (counts against timeout)
- if no response within timeout, **return an empty collection immediately**
2. if there is fetch data available, **return it immediately**
3. if there is no fetch request in flight, send fetch requests
4. poll for fetch responses (counts against timeout)
- if no response within timeout, **return an empty collection** (leaving async fetch request for the next poll)
- if we get a response, **return the response**
The old method, `poll(long timeout)` is deprecated, but we do not change its semantics, so it remains:
1. iff a metadata update is needed:
1. send (asynchronous) metadata requests
2. poll for metadata responses *indefinitely until we get it*
2. if there is fetch data available, **return it immediately**
3. if there is no fetch request in flight, send fetch requests
4. poll for fetch responses (counts against timeout)
- if no response within timeout, **return an empty collection** (leaving async fetch request for the next poll)
- if we get a response, **return the response**
One notable usage is prohibited by the new `poll`: previously, you could call `poll(0)` to block for metadata updates, for example to initialize the client, supposedly without fetching records. Note, though, that this behavior is not according to any contract, and there is no guarantee that `poll(0)` won't return records the first time it's called. Therefore, it has always been unsafe to ignore the response.
Add the new stricter-timeout version of
pollproposed in KIP-266.The pre-existing variant
poll(long timeout)would block indefinitely for metadataupdates if they were needed, then it would issue a fetch and poll for
timeoutmsfor new records. The initial indefinite metadata block caused applications to become
stuck when the brokers became unavailable. The existence of the timeout parameter
made the indefinite block especially unintuitive.
This PR adds
poll(Duration timeout)with the semantics:The old method,
poll(long timeout)is deprecated, but we do not change its semantics, so it remains:One notable usage is prohibited by the new
poll: previously, you could callpoll(0)to block for metadata updates, for example to initialize the client, supposedly without fetching records. Note, though, that this behavior is not according to any contract, and there is no guarantee thatpoll(0)won't return records the first time it's called. Therefore, it has always been unsafe to ignore the response.Committer Checklist (excluded from commit message)