showuon commented on a change in pull request #11433:
URL: https://github.com/apache/kafka/pull/11433#discussion_r820068525
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
##########
@@ -343,6 +382,16 @@ public void handleAssignment(final Map<TaskId,
Set<TopicPartition>> activeTasks,
}
}
+ // We commit active tasks only if there are any active tasks which
need restoring.
+ if (!activeTasksNeedCommit.isEmpty()) {
+ final AtomicReference<RuntimeException>
activeTasksCommitException = new AtomicReference<>(null);
+ commitActiveTasks(activeTasksNeedCommit,
activeTasksCommitException);
+
+ if (activeTasksCommitException.get() != null) {
+ throw activeTasksCommitException.get();
+ }
+
Review comment:
nit: extra line
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
##########
@@ -265,6 +265,41 @@ private void closeDirtyAndRevive(final Collection<Task>
taskWithChangelogs, fina
}
}
+ private void commitActiveTasks(final Set<Task> activeTasksNeedCommit,
final AtomicReference<RuntimeException> activeTasksCommitException) {
+
+ final Map<Task, Map<TopicPartition, OffsetAndMetadata>>
consumedOffsetsPerTask = new HashMap<>();
+ prepareCommitAndAddOffsetsToMap(activeTasksNeedCommit,
consumedOffsetsPerTask);
+
+ final Set<Task> dirtyTasks = new HashSet<>();
+ try {
+ commitOffsetsOrTransaction(consumedOffsetsPerTask);
+ } catch (final TaskCorruptedException e) {
+ log.warn("Some tasks were corrupted when trying to commit offsets,
these will be cleaned and revived: {}",
+ e.corruptedTasks());
+
+ // If we hit a TaskCorruptedException it must be EOS, just handle
the cleanup for those corrupted tasks right here
+ dirtyTasks.addAll(tasks.tasks(e.corruptedTasks()));
+ closeDirtyAndRevive(dirtyTasks, true);
+ } catch (final RuntimeException e) {
+ log.error("Exception caught while committing active tasks " +
activeTasksNeedCommit, e);
Review comment:
Should we output the `consumedOffsetsPerTask.keySet()` instead of
`activeTasksNeedCommit` here?
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
##########
@@ -265,6 +265,41 @@ private void closeDirtyAndRevive(final Collection<Task>
taskWithChangelogs, fina
}
}
+ private void commitActiveTasks(final Set<Task> activeTasksNeedCommit,
final AtomicReference<RuntimeException> activeTasksCommitException) {
+
+ final Map<Task, Map<TopicPartition, OffsetAndMetadata>>
consumedOffsetsPerTask = new HashMap<>();
+ prepareCommitAndAddOffsetsToMap(activeTasksNeedCommit,
consumedOffsetsPerTask);
+
+ final Set<Task> dirtyTasks = new HashSet<>();
+ try {
+ commitOffsetsOrTransaction(consumedOffsetsPerTask);
+ } catch (final TaskCorruptedException e) {
+ log.warn("Some tasks were corrupted when trying to commit offsets,
these will be cleaned and revived: {}",
+ e.corruptedTasks());
+
+ // If we hit a TaskCorruptedException it must be EOS, just handle
the cleanup for those corrupted tasks right here
+ dirtyTasks.addAll(tasks.tasks(e.corruptedTasks()));
+ closeDirtyAndRevive(dirtyTasks, true);
+ } catch (final RuntimeException e) {
+ log.error("Exception caught while committing active tasks " +
activeTasksNeedCommit, e);
Review comment:
nit:
`log.error("Exception caught while committing active tasks: {}",
activeTasksNeedCommit, e);`
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
##########
@@ -265,6 +265,41 @@ private void closeDirtyAndRevive(final Collection<Task>
taskWithChangelogs, fina
}
}
+ private void commitActiveTasks(final Set<Task> activeTasksNeedCommit,
final AtomicReference<RuntimeException> activeTasksCommitException) {
+
+ final Map<Task, Map<TopicPartition, OffsetAndMetadata>>
consumedOffsetsPerTask = new HashMap<>();
+ prepareCommitAndAddOffsetsToMap(activeTasksNeedCommit,
consumedOffsetsPerTask);
+
+ final Set<Task> dirtyTasks = new HashSet<>();
+ try {
+ commitOffsetsOrTransaction(consumedOffsetsPerTask);
+ } catch (final TaskCorruptedException e) {
+ log.warn("Some tasks were corrupted when trying to commit offsets,
these will be cleaned and revived: {}",
+ e.corruptedTasks());
+
+ // If we hit a TaskCorruptedException it must be EOS, just handle
the cleanup for those corrupted tasks right here
+ dirtyTasks.addAll(tasks.tasks(e.corruptedTasks()));
+ closeDirtyAndRevive(dirtyTasks, true);
+ } catch (final RuntimeException e) {
+ log.error("Exception caught while committing active tasks " +
activeTasksNeedCommit, e);
+ activeTasksCommitException.compareAndSet(null, e);
+ dirtyTasks.addAll(consumedOffsetsPerTask.keySet());
+ }
+
+ // we enforce checkpointing upon suspending a task: if it is resumed
later we just proceed normally, if it is
+ // going to be closed we would checkpoint by then
Review comment:
I don't think the comment here is correct. We didn't `suspend` any task
in `handleAssignment`, right? Based on the comments
[here](https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java#L596-L599),
it looks like we don't expect to force checkpoint in RUNNING state. So, I'm
not quite sure we should force all checkpoint here. Thoughts?
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
##########
@@ -265,6 +265,41 @@ private void closeDirtyAndRevive(final Collection<Task>
taskWithChangelogs, fina
}
}
+ private void commitActiveTasks(final Set<Task> activeTasksNeedCommit,
final AtomicReference<RuntimeException> activeTasksCommitException) {
+
+ final Map<Task, Map<TopicPartition, OffsetAndMetadata>>
consumedOffsetsPerTask = new HashMap<>();
+ prepareCommitAndAddOffsetsToMap(activeTasksNeedCommit,
consumedOffsetsPerTask);
+
+ final Set<Task> dirtyTasks = new HashSet<>();
+ try {
+ commitOffsetsOrTransaction(consumedOffsetsPerTask);
+ } catch (final TaskCorruptedException e) {
+ log.warn("Some tasks were corrupted when trying to commit offsets,
these will be cleaned and revived: {}",
+ e.corruptedTasks());
+
+ // If we hit a TaskCorruptedException it must be EOS, just handle
the cleanup for those corrupted tasks right here
+ dirtyTasks.addAll(tasks.tasks(e.corruptedTasks()));
+ closeDirtyAndRevive(dirtyTasks, true);
+ } catch (final RuntimeException e) {
+ log.error("Exception caught while committing active tasks " +
activeTasksNeedCommit, e);
+ activeTasksCommitException.compareAndSet(null, e);
+ dirtyTasks.addAll(consumedOffsetsPerTask.keySet());
+ }
+
+ // we enforce checkpointing upon suspending a task: if it is resumed
later we just proceed normally, if it is
+ // going to be closed we would checkpoint by then
+ for (final Task task : activeTasksNeedCommit) {
+ if (!dirtyTasks.contains(task)) {
+ try {
+ task.postCommit(true);
+ } catch (final RuntimeException e) {
+ log.error("Exception caught while post-committing task " +
task.id(), e);
Review comment:
nit:
`log.error("Exception caught while post-committing task: {}", task.id(), e);`
##########
File path:
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
##########
@@ -265,6 +265,41 @@ private void closeDirtyAndRevive(final Collection<Task>
taskWithChangelogs, fina
}
}
+ private void commitActiveTasks(final Set<Task> activeTasksNeedCommit,
final AtomicReference<RuntimeException> activeTasksCommitException) {
+
+ final Map<Task, Map<TopicPartition, OffsetAndMetadata>>
consumedOffsetsPerTask = new HashMap<>();
+ prepareCommitAndAddOffsetsToMap(activeTasksNeedCommit,
consumedOffsetsPerTask);
+
+ final Set<Task> dirtyTasks = new HashSet<>();
+ try {
+ commitOffsetsOrTransaction(consumedOffsetsPerTask);
Review comment:
Looks like the code has been refactored in this PR:
https://github.com/apache/kafka/pull/11738
We have to call it via `taskExecutor`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]