KAFKA-10150: task state transitions/management and committing cleanup#8856
Conversation
|
test this please |
|
Also failed with unrelated (extremely) flaky |
guozhangwang
left a comment
There was a problem hiding this comment.
Have a few comments, but otherwise looks great to me.
@ableegoldman lmk if you think the test failures are transient, I can re-trigger it.
mjsax
left a comment
There was a problem hiding this comment.
Thanks for the fix @ableegoldman!
|
test this please |
1 similar comment
|
test this please |
vvcephei
left a comment
There was a problem hiding this comment.
Thanks for this @ableegoldman ! I only had time for a partial review. Here's what I thought so far.
There was a problem hiding this comment.
I get that standbys should never really be in RESTORING state, but it still doesn't seem like it's philosophically any more illegal to suspend from RESTORING than it is from RUNNING. I'd vote to legalize RESTORING here. It does seem like a useful sanity check for CLOSED to be illegal, though.
There was a problem hiding this comment.
I agree 100%, but at some point in the past we started checking for RESTORING and throwing IllegalStateException all over StandbyTask. I wanted to keep the changes here to a minimum and figured we should at least be consistent with the current pattern elsewhere
There was a problem hiding this comment.
This was a test I added a little while back in response to a bugfix, but it no longer makes sense in the current context (in fact it's currently not really testing anything at all, since the original point was to make sure the changelog reader partitions were cleaned up but that's not even the responsibility of the TaskManager anymore)
|
Retest this please. |
2 similar comments
|
Retest this please. |
|
Retest this please. |
|
Retest this please. |
1 similar comment
|
Retest this please. |
7b86055 to
080a106
Compare
|
Retest this please. |
1 similar comment
|
Retest this please. |
|
Mr.J. does not like me recently... Will retry later. |
|
Haha. After complaining Mr.J. changes his mind :D |
|
The sad thing is it's doomed to fail ( |
There was a problem hiding this comment.
So far, we did not allow idempotent state transitions in the state machine itself, but handle it caller side. -- It seem inconsistent to allow SUSPENDED -> SUSPEND but not CREATE -> CREATED etc.
I would recommend to keep the current pattern and avoid calling transiteState() if the task is already in the target state. -- I would also be happy to change it, but for this case, we should change it for all cases. However, this would enlarge the scope of this PR and I think it better not to do it in this PR.
There was a problem hiding this comment.
I see. I was just thinking we should make the idempotency explicit for each state by allowing/disallowing the transition, but I agree we can do that in a followup PR
There was a problem hiding this comment.
Why remove the < arrow? We can still transit from RESTORING to SUSPENDED.
Super-nit: +---->| Suspended -> +---> | Suspended
There was a problem hiding this comment.
The diff makes it hard to tell, but I "merged" the path to SUSPENDED from CREATED and RESTORING. I find it a bit easier to follow when all the arrows are unidirectional
There was a problem hiding this comment.
If we hit an exception in handleRevocation why would we continue here? Are we still in a "clean enough" state to actually continue?
Below we call completeTaskCloseClean(task) what seem incorrect for this case as it might close clean task even if we did not successfully commit before.
There was a problem hiding this comment.
If we hit an exception in handleRevocation on some task then we would skip out on suspending the rest of the tasks, ie the set of not-suspended tasks does not contain the task that threw (of course if one task threw an exception then its likely others will too, but not guaranteed).
But maybe it's cleaner to catch exceptions during handleRevocation and at least make sure every task gets suspended? I'll try that
On a related note, if we always have to commit before closing (or at least attempt to), should we just remove the writeCheckpointIfNeeded call from closeClean? Seems like the pre/postCommit should be responsible for whether to checkpoint, not close. In this case, it's completely fine to attempt a clean close of a dirty task, as the closeClean method will just maybe throw in which case we can close dirty. WDYT?
There was a problem hiding this comment.
Instead of checkpointing, we can check if clean && commitNeeded & checkpoint != null and then throw an exception on closeClean (which would result in calling closeDirty)
There was a problem hiding this comment.
I re-read the current code structure and got some questions:
- we collect checkpoint from
prepareCommitand check if it is not null inpostCommit, but the actual checkpoint value itself is always collectable post the commit, and hence what's only required to that we need to know if we need to write a checkpoint file or not. Previously this needs to be decided since we may transit the state in between but now from the source code it seems to me that we would only callprepare/postbefore suspend / close ever, so this is no longer required actually, i.e. we can decide whether we need to checkpoint and then collect the checkpoint map and write the file if needed in a single call. Is that right?
- I think I agree with you that it is cleaner to make sure in
handleRevocation, we still transit those revoked partition's corresponding tasks to suspended even if some of their commit call failed.
There was a problem hiding this comment.
- I think you're right, we don't need to keep track of the current
checkpointoffsets at all and can just write the currentcheckpointableOffsetsinpostCommit - done
There was a problem hiding this comment.
postCommit only writes a checkpoint for non-eos. Thus, we still need to write a checkpoint in close() for the eos-case (if just blindly for all cases as we do atm).
There was a problem hiding this comment.
postCommit will always write the checkpoint if the task is in SUSPENDED, which it should always be before being closed
There was a problem hiding this comment.
Why do we do the try-catch as outer-layer? In an exception occurs, we should stop looping through the tasks to call postCommit() -- is this intended? If yes, why?
There was a problem hiding this comment.
Well if commit throws an exception, then we shouldn't call postCommit right?
Or are you saying if commit succeeds but postCommit throws for one task, we should still loop through and try to postCommit all the other tasks?
There was a problem hiding this comment.
Yeah I think if the actual consumer.commit call failed, then we should not trigger postCommit for any one.
As for postCommit, I think it should never fail (we swallow the IO exception happened, because for non-EOS it is just fine, for EOS we would bootstrap from scratch).
There was a problem hiding this comment.
I meant the later. And I agree that if commit fails, we should not call postCommit().
For failure in postCommit: we make assumptions about the current code what seems dangerous (ie, not future prove)? -- IMHO, if postCommit fails, we need to close the corresponding task dirty and either recreate it, or rebalance, but we should also continue to call postCommit() for all other tasks?
There was a problem hiding this comment.
I see. Then I think it makes sense to always attempt to write the checkpoint/call postCommit for a task that was successfully committed, regardless of whether something went wrong during postCommit with a different task
And I agree, we should not make assumptions about the current code not throwing, unless it's explicitly in the contract of the method that it will never throw (which is not the case for postCommit
There was a problem hiding this comment.
Sounds good, in that case the nested try-catch would be necessary.
guozhangwang
left a comment
There was a problem hiding this comment.
Made another pass on the PR, left some meta comments.
There was a problem hiding this comment.
I re-read the current code structure and got some questions:
- we collect checkpoint from
prepareCommitand check if it is not null inpostCommit, but the actual checkpoint value itself is always collectable post the commit, and hence what's only required to that we need to know if we need to write a checkpoint file or not. Previously this needs to be decided since we may transit the state in between but now from the source code it seems to me that we would only callprepare/postbefore suspend / close ever, so this is no longer required actually, i.e. we can decide whether we need to checkpoint and then collect the checkpoint map and write the file if needed in a single call. Is that right?
- I think I agree with you that it is cleaner to make sure in
handleRevocation, we still transit those revoked partition's corresponding tasks to suspended even if some of their commit call failed.
There was a problem hiding this comment.
Yeah I think if the actual consumer.commit call failed, then we should not trigger postCommit for any one.
As for postCommit, I think it should never fail (we swallow the IO exception happened, because for non-EOS it is just fine, for EOS we would bootstrap from scratch).
ableegoldman
left a comment
There was a problem hiding this comment.
Addressed all current comments, ready for testing & final review
There was a problem hiding this comment.
- I think you're right, we don't need to keep track of the current
checkpointoffsets at all and can just write the currentcheckpointableOffsetsinpostCommit - done
f6cbdad to
292cfbd
Compare
|
Retest this please. |
|
test this |
1 similar comment
|
test this |
There was a problem hiding this comment.
I think we still need to make this call -- in eager rebalancing, we suspend a task when we get a partition revoked. For this case, we "forget" the current offset within the consumer and thus need to clear the partition grouper. Otherwise, we might read the data a second time, if the partition is reassigned (what would violate EOS).
There was a problem hiding this comment.
I see. So we should only clear it here, and not in close
Just curious, why do we "forget" the current offset? I mean, haven't we just committed the current offset before suspending (and if that failed we would close all tasks right away). Maybe I'm misunderstanding what you mean
There was a problem hiding this comment.
The consumer tracks offset internal, however, we buffer data in our internal queue. Thus, the offset tracked by the consumer, might be larger than the offset we commit (we take the offset we commit not from the consumer, but it's based on the records we did take out of the queue and processed).
In eager rebalancing, the consumer clears its internal state if a partition in revoked (and we only suspend the task), including the tracked offsets. If the partition in re-assigned, the consumer fetches the last committed offset to start fetching. Thus, if we don't clear the queue, we might fetch same data that is already in the queue a second time.
Does this make sense?
There was a problem hiding this comment.
Ah that's a good catch. Makes sense to me.
There was a problem hiding this comment.
Got it, thanks for the explanation. I'll move it back to postCommit with a note
There was a problem hiding this comment.
I'm not 100% certain that the Consumer does clear its internal buffer on revocation. At least, I couldn't find it in the code, but maybe I'm looking in the wrong place.
Not arguing we shouldn't clear the partition group here, was just wondering about this for my own sake. Hm
There was a problem hiding this comment.
Not 100% familiar with the consumer code, but in SubscriptionState#assignFromSubscribed new TopicPartitionState are created with position = null.
There was a problem hiding this comment.
Ah that seems right. Thanks!
There was a problem hiding this comment.
Thinking about punctuation, should we actually call commitOffsetsOrTransaction() unconditionally (ie, not consider if consumedOffsetsAndMetadataPerTask is empty or not?
We can still move the check inside consumedOffsetsAndMetadataPerTask, but for EOS there might pending writes from punctuation that we still need to commit?
This would apply to all calls of commitOffsetsOrTransaction ?
There was a problem hiding this comment.
Hm. So in the punctuation case -- where commitNeeded is true but consumedOffsets is empty -- we still need to call commitOffsetsOrTransaction (and postCommit) because the punctuation may for example write to a state store and generate changelog records. So we would need to commit that transaction, and also write the checkpoint file.
Makes sense
There was a problem hiding this comment.
Committable offsets here should contain consumed offsets, and punctuation itself should never update those consumed offsets right?
I think we can skip the call if consumedOffsetsAndMetadataPerTask is empty.
There was a problem hiding this comment.
Committable offsets here should contain consumed offsets, and punctuation itself should never update those consumed offsets right
Yes.
I think we can skip the call if consumedOffsetsAndMetadataPerTask is empty.
For non-eos, yes, because for non-eos commitOffsetsOrTransaction() would only commit offsets via the consumer (this can be skipped if empty). However, for eos (alpha and beta), we might have a pending transaction that we need to commit on the producer, too.
78274fe to
3ff79ea
Compare
|
Retest this please. |
1 similar comment
|
Retest this please. |
| if (task.commitNeeded()) { | ||
| tasksToCommit.add(task); | ||
| final Map<TopicPartition, OffsetAndMetadata> committableOffsets = task.prepareCommit(); | ||
| if (task.isActive()) { |
There was a problem hiding this comment.
As @mjsax pointed out, we should still commit even if there are no consumed offsets. However, we should not commit the offsets/transaction if there are no active tasks that need committing
|
Failures were all due to known-flaky/completely broken |
…#8856) * KAFKA-10150: always transition to SUSPENDED during suspend, no matter the current state only call prepareCommit before closing if task.commitNeeded is true * Don't commit any consumed offsets during handleAssignment -- revoked active tasks (and any others that need committing) will be committed during handleRevocation so we only need to worry about cleaning them up in handleAssignment * KAFKA-10152: when recycling a task we should always commit consumed offsets (if any), but don't need to write the checkpoint (since changelog offsets are preserved across task transitions) * Make sure we close all tasks during shutdown, even if an exception is thrown during commit Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
|
Cherry-picked to 2.6. |
* 'trunk' of github.com:apache/kafka: KAFKA-10150: task state transitions/management and committing cleanup (apache#8856) KAFKA-10169: Error message when transit to Aborting / AbortableError / FatalError (apache#8880) KAFKA-9974: Fix flaky test by removing unneeded asserts (apache#8646) MINOR: Documentation for KIP-585 (apache#8839)
suspend, no matter the current stateprepareCommitbefore closing iftask.commitNeededis truehandleAssignment-- revoked active tasks (and any others that need committing) will be committed duringhandleRevocationso we only need to worry about cleaning them up inhandleAssignmentcheckpointableOffsetsbefore flushing inprepareCommit(This was the root cause of KAFKA-10151 -- we now don't save the offsets at all duringprepareCommitand just write the current offsets duringpostCommitMust be cherry-picked to 2.6