-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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-6145: Pt 1. Bump protocol version and encode task lag map #8121
Changes from all commits
5a67b59
8b45f78
b4d91a8
2aabf29
f510057
c063dc4
424af9a
cac1f39
72ecf58
c2da9f7
a37a8eb
40e20e3
7d8bdc2
e4e941f
0fe088a
361e449
951c245
41d70a2
aeebfa8
16be046
cce6847
36b94cc
f0f2022
9c1849c
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 |
---|---|---|
|
@@ -51,6 +51,7 @@ | |
|
||
import static org.apache.kafka.streams.processor.internals.Task.State.CREATED; | ||
import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; | ||
import static org.apache.kafka.streams.processor.internals.Task.State.RUNNING; | ||
|
||
public class TaskManager { | ||
// initialize the task list | ||
|
@@ -354,11 +355,27 @@ void handleLostAll() { | |
} | ||
} | ||
|
||
/** | ||
* @return Map from task id to its total offset summed across all state stores | ||
*/ | ||
public Map<TaskId, Long> getTaskOffsetSums() { | ||
final Map<TaskId, Long> taskOffsetSums = new HashMap<>(); | ||
|
||
for (final TaskId id : tasksOnLocalStorage()) { | ||
if (isRunning(id)) { | ||
taskOffsetSums.put(id, Task.LATEST_OFFSET); | ||
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'm just a tiny bit uncomfortable with re-using that sentinel, because the correctness of our logic depends on the active sentinel being less than the standby sentinel, so it must be less than zero. Do we have a reason to believe that 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 actually changed this based on working on the next PR, as |
||
} else { | ||
taskOffsetSums.put(id, 0L); | ||
} | ||
} | ||
return taskOffsetSums; | ||
} | ||
|
||
/** | ||
* Returns ids of tasks whose states are kept on the local storage. This includes active, standby, and previously | ||
* assigned but not yet cleaned up tasks | ||
*/ | ||
public Set<TaskId> tasksOnLocalStorage() { | ||
private Set<TaskId> tasksOnLocalStorage() { | ||
// A client could contain some inactive tasks whose states are still kept on the local storage in the following scenarios: | ||
// 1) the client is actively maintaining standby tasks by maintaining their states from the change log. | ||
// 2) the client has just got some tasks migrated out of itself to other clients while these task states | ||
|
@@ -472,6 +489,11 @@ private Stream<Task> standbyTaskStream() { | |
return tasks.values().stream().filter(t -> !t.isActive()); | ||
} | ||
|
||
private boolean isRunning(final TaskId id) { | ||
final Task task = tasks.get(id); | ||
return task != null && task.isActive() && task.state() == RUNNING; | ||
} | ||
|
||
/** | ||
* @throws TaskMigratedException if committing offsets failed (non-EOS) | ||
* or if the task producer got fenced (EOS) | ||
|
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.
This plus the tech debt cleanup allows for the subscription handling to be greatly simplified, here and below in #assign