-
Notifications
You must be signed in to change notification settings - Fork 14k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-8179: Part 7, cooperative rebalancing in Streams #7386
Changes from all commits
84fccef
9d0bc33
bb7961b
22f90bf
79075be
ac0018f
a90ff4c
c111155
47d291a
a06deaf
0f402cf
da6d6c3
393d207
8c10ee3
c19aa66
7e15df2
34b5368
f047cd9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -119,18 +119,6 @@ public void commit() { | |
commitNeeded = false; | ||
} | ||
|
||
/** | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we remove the standby task close? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We remove the |
||
* <pre> | ||
* - flush store | ||
* - checkpoint store | ||
* </pre> | ||
*/ | ||
@Override | ||
public void suspend() { | ||
log.debug("Suspending"); | ||
flushAndCheckpointState(); | ||
} | ||
|
||
private void flushAndCheckpointState() { | ||
stateMgr.flush(); | ||
stateMgr.checkpoint(Collections.emptyMap()); | ||
|
@@ -162,13 +150,6 @@ public void close(final boolean clean, | |
taskClosed = true; | ||
} | ||
|
||
@Override | ||
public void closeSuspended(final boolean clean, | ||
final boolean isZombie, | ||
final RuntimeException e) { | ||
close(clean, isZombie); | ||
} | ||
|
||
/** | ||
* Updates a state store using records from one change log partition | ||
* | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -560,7 +560,6 @@ private void initTopology() { | |
* @throws TaskMigratedException if committing offsets failed (non-EOS) | ||
* or if the task producer got fenced (EOS) | ||
*/ | ||
@Override | ||
public void suspend() { | ||
log.debug("Suspending"); | ||
suspend(true, false); | ||
|
@@ -674,10 +673,7 @@ private void closeTopology() { | |
} | ||
|
||
// helper to avoid calling suspend() twice if a suspended task is not reassigned and closed | ||
@Override | ||
public void closeSuspended(final boolean clean, | ||
final boolean isZombie, | ||
RuntimeException firstException) { | ||
void closeSuspended(final boolean clean, RuntimeException firstException) { | ||
try { | ||
closeStateManager(clean); | ||
} catch (final RuntimeException e) { | ||
|
@@ -729,7 +725,7 @@ public void close(boolean clean, | |
log.error("Could not close task due to the following error:", e); | ||
} | ||
|
||
closeSuspended(clean, isZombie, firstException); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nice cleanup! |
||
closeSuspended(clean, firstException); | ||
|
||
taskClosed = true; | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -134,13 +134,13 @@ public class StreamThread extends Thread { | |
*/ | ||
public enum State implements ThreadStateTransitionValidator { | ||
|
||
CREATED(1, 5), // 0 | ||
STARTING(2, 3, 5), // 1 | ||
PARTITIONS_REVOKED(3, 5), // 2 | ||
PARTITIONS_ASSIGNED(2, 3, 4, 5), // 3 | ||
RUNNING(2, 3, 5), // 4 | ||
PENDING_SHUTDOWN(6), // 5 | ||
DEAD; // 6 | ||
CREATED(1, 5), // 0 | ||
STARTING(2, 3, 5), // 1 | ||
PARTITIONS_REVOKED(2, 3, 5), // 2 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Allow PARTITIONS_REVOKED to transition to itself (but callback is a no-op if no new partitions have been revoked) |
||
PARTITIONS_ASSIGNED(2, 3, 4, 5), // 3 | ||
RUNNING(2, 3, 5), // 4 | ||
PENDING_SHUTDOWN(6), // 5 | ||
DEAD; // 6 | ||
|
||
private final Set<Integer> validTransitions = new HashSet<>(); | ||
|
||
|
@@ -744,9 +744,9 @@ void runOnce() { | |
// to unblock the restoration as soon as possible | ||
records = pollRequests(Duration.ZERO); | ||
} else if (state == State.PARTITIONS_REVOKED) { | ||
// try to fetch some records with normal poll time | ||
// in order to wait long enough to get the join response | ||
records = pollRequests(pollTime); | ||
// try to fetch som records with zero poll millis to unblock | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can potentially be in PARTITIONS_REVOKED with cooperative rebalancing so we don't want to just block doing nothing during the rebalance -- not sure if it's worth polling for zero since this is rare with cooperative, or some other time < There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In EAGER, during the PARTITIONS_REVOKED state we would not return any data from consumer anyways; In future COOPERATIVE, even if we can return some data during the rebalance, the transition of PARTITIONS_REVOKED -> PARTITIONS_ASSIGNED would happen in a single Also note that in my PR for returning data in the middle of a rebalance, we still pass in non-zero timeout for finding the coordinator so that we are ensured to have one round-trip at least within that call. |
||
// other useful work while waiting for the join response | ||
records = pollRequests(Duration.ZERO); | ||
} else if (state == State.RUNNING || state == State.STARTING) { | ||
// try to fetch some records with normal poll time | ||
// in order to get long polling | ||
|
@@ -980,7 +980,12 @@ boolean maybeCommit() { | |
} | ||
} | ||
|
||
lastCommitMs = now; | ||
if (committed == -1) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is a meta comment: maybe another (equally hacky?) way to do this, is to expose the As for now I think this way is fine, cannot really think of a better way that does not change public APIs. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. cc @abbccdda There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What's this trace for? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I thought it might be helpful to see that we are not committing some of what we've processed, because it is we can't commit during a rebalance. But I don't think it's absolutely necessary and can take it out if you don't think it adds much? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it's okay to leave it as TRACE, as practically we do not turn on TRACE that frequently. |
||
log.trace("Unable to commit as we are in the middle of a rebalance, will try again when it completes."); | ||
} else { | ||
lastCommitMs = now; | ||
} | ||
|
||
processStandbyRecords = true; | ||
} else { | ||
committed = taskManager.maybeCommitActiveTasksPerUserRequested(); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice cleanup! In #5501 we moved the close / abort logic into the caller suspend function, and we should actually remove the parameter in that PR but overlooked it.