-
Notifications
You must be signed in to change notification settings - Fork 14.8k
KAFKA-12523: handle TaskCorruption/TimeoutException during handleCorruption and handleRevocation #10407
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-12523: handle TaskCorruption/TimeoutException during handleCorruption and handleRevocation #10407
Changes from all commits
8dc0f50
549a5f7
a843f11
dbec22d
4ce51e9
7282884
5950b39
d6f8c0b
1a579a4
4d4f41c
dc91e75
1446ec2
9c8730a
723520b
e5cca19
4a9ffff
669367f
4d23c99
0d21ff4
ecb7e7c
9ea3256
f6361c0
4aed51d
9db590b
4c78a79
55c076b
6fcdd56
fa4cde5
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 |
|---|---|---|
|
|
@@ -155,40 +155,55 @@ void handleRebalanceComplete() { | |
| * @throws TaskMigratedException | ||
| */ | ||
| void handleCorruption(final Set<TaskId> corruptedTasks) { | ||
| final Map<Task, Collection<TopicPartition>> corruptedStandbyTasks = new HashMap<>(); | ||
| final Map<Task, Collection<TopicPartition>> corruptedActiveTasks = new HashMap<>(); | ||
| final Set<Task> corruptedActiveTasks = new HashSet<>(); | ||
| final Set<Task> corruptedStandbyTasks = new HashSet<>(); | ||
|
|
||
| for (final TaskId taskId : corruptedTasks) { | ||
| final Task task = tasks.task(taskId); | ||
| if (task.isActive()) { | ||
| corruptedActiveTasks.put(task, task.changelogPartitions()); | ||
| corruptedActiveTasks.add(task); | ||
|
Contributor
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. Thanks for the cleanup! I think in the past we may only mark some subset of changelog partitions as corrupted, but later we would always just mark all of them as corrupted. Just following that thought, maybe in
Member
Author
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 didn't want to go all the way with consolidating this logic since eventually we may want to have it so that only the subset of partitions/stores which are actually corrupted will need to be wiped out. So I'd prefer to leave this as-is for now and keep the places in which we infer the changelogs from the task restricted to just the TaskManager for now |
||
| } else { | ||
| corruptedStandbyTasks.put(task, task.changelogPartitions()); | ||
| corruptedStandbyTasks.add(task); | ||
| } | ||
| } | ||
|
|
||
| // Make sure to clean up any corrupted standby tasks in their entirety before committing | ||
| // since TaskMigrated can be thrown and the resulting handleLostAll will only clean up active tasks | ||
| closeAndRevive(corruptedStandbyTasks); | ||
|
|
||
| commit(tasks() | ||
ableegoldman marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| .values() | ||
| .stream() | ||
| .filter(t -> t.state() == Task.State.RUNNING || t.state() == Task.State.RESTORING) | ||
| .filter(t -> !corruptedTasks.contains(t.id())) | ||
| .collect(Collectors.toSet()) | ||
| ); | ||
| closeDirtyAndRevive(corruptedStandbyTasks, true); | ||
|
|
||
| // We need to commit before closing the corrupted active tasks since this will force the ongoing txn to abort | ||
| try { | ||
| commitAndFillInConsumedOffsetsAndMetadataPerTaskMap(tasks() | ||
| .values() | ||
| .stream() | ||
| .filter(t -> t.state() == Task.State.RUNNING || t.state() == Task.State.RESTORING) | ||
| .filter(t -> !corruptedTasks.contains(t.id())) | ||
| .collect(Collectors.toSet()), | ||
| new HashMap<>() | ||
| ); | ||
| } catch (final TaskCorruptedException e) { | ||
ableegoldman marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| log.info("Some additional tasks were found corrupted while trying to commit, these will be added to the " + | ||
| "tasks to clean and revive: {}", e.corruptedTasks()); | ||
| corruptedActiveTasks.addAll(tasks.tasks(e.corruptedTasks())); | ||
| } catch (final TimeoutException e) { | ||
| log.info("Hit TimeoutException when committing all non-corrupted tasks, these will be closed and revived"); | ||
| final Collection<Task> uncorruptedTasks = new HashSet<>(tasks.activeTasks()); | ||
| uncorruptedTasks.removeAll(corruptedActiveTasks); | ||
| // Those tasks which just timed out can just be closed dirty without marking changelogs as corrupted | ||
| closeDirtyAndRevive(uncorruptedTasks, false); | ||
|
Contributor
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. If
Member
Author
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. It seems we guarantee that |
||
| } | ||
|
|
||
| closeAndRevive(corruptedActiveTasks); | ||
| closeDirtyAndRevive(corruptedActiveTasks, true); | ||
| } | ||
|
|
||
| private void closeAndRevive(final Map<Task, Collection<TopicPartition>> taskWithChangelogs) { | ||
| for (final Map.Entry<Task, Collection<TopicPartition>> entry : taskWithChangelogs.entrySet()) { | ||
| final Task task = entry.getKey(); | ||
| private void closeDirtyAndRevive(final Collection<Task> taskWithChangelogs, final boolean markAsCorrupted) { | ||
| for (final Task task : taskWithChangelogs) { | ||
| final Collection<TopicPartition> corruptedPartitions = task.changelogPartitions(); | ||
|
|
||
| // mark corrupted partitions to not be checkpointed, and then close the task as dirty | ||
| final Collection<TopicPartition> corruptedPartitions = entry.getValue(); | ||
| task.markChangelogAsCorrupted(corruptedPartitions); | ||
| if (markAsCorrupted) { | ||
ableegoldman marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| task.markChangelogAsCorrupted(corruptedPartitions); | ||
| } | ||
|
|
||
| try { | ||
| // we do not need to take the returned offsets since we are not going to commit anyways; | ||
|
|
@@ -201,8 +216,11 @@ private void closeAndRevive(final Map<Task, Collection<TopicPartition>> taskWith | |
|
|
||
| try { | ||
| task.suspend(); | ||
|
|
||
| // we need to enforce a checkpoint that removes the corrupted partitions | ||
| task.postCommit(true); | ||
| if (markAsCorrupted) { | ||
| task.postCommit(true); | ||
| } | ||
| } catch (final RuntimeException swallow) { | ||
| log.error("Error suspending corrupted task {} ", task.id(), swallow); | ||
| } | ||
|
|
@@ -332,8 +350,8 @@ private void handleCloseAndRecycle(final Set<Task> tasksToRecycle, | |
| // write the checkpoint file. | ||
| final Map<TopicPartition, OffsetAndMetadata> offsets = task.prepareCommit(); | ||
| if (!offsets.isEmpty()) { | ||
| log.error("Task {} should has been committed when it was suspended, but it reports non-empty " + | ||
| "offsets {} to commit; it means it fails during last commit and hence should be closed dirty", | ||
| log.error("Task {} should have been committed when it was suspended, but it reports non-empty " + | ||
| "offsets {} to commit; this means it failed during last commit and hence should be closed dirty", | ||
| task.id(), offsets); | ||
|
|
||
| tasksToCloseDirty.add(task); | ||
|
|
@@ -509,30 +527,47 @@ void handleRevocation(final Collection<TopicPartition> revokedPartitions) { | |
| prepareCommitAndAddOffsetsToMap(commitNeededActiveTasks, consumedOffsetsPerTask); | ||
| } | ||
|
|
||
| // even if commit failed, we should still continue and complete suspending those tasks, | ||
| // so we would capture any exception and throw | ||
| // even if commit failed, we should still continue and complete suspending those tasks, so we would capture | ||
| // any exception and rethrow it at the end. some exceptions may be handled immediately and then swallowed, | ||
| // as such we just need to skip those dirty tasks in the checkpoint | ||
| 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 TimeoutException e) { | ||
| log.warn("Timed out while trying to commit all tasks during revocation, these will be cleaned and revived"); | ||
|
|
||
| // If we hit a TimeoutException it must be ALOS, just close dirty and revive without wiping the state | ||
ableegoldman marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| dirtyTasks.addAll(consumedOffsetsPerTask.keySet()); | ||
| closeDirtyAndRevive(dirtyTasks, false); | ||
| } catch (final RuntimeException e) { | ||
| log.error("Exception caught while committing those revoked tasks " + revokedActiveTasks, e); | ||
| firstException.compareAndSet(null, e); | ||
| dirtyTasks.addAll(consumedOffsetsPerTask.keySet()); | ||
| } | ||
|
|
||
| // only try to complete post-commit if committing succeeded; | ||
| // 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 | ||
| if (firstException.get() == null) { | ||
| for (final Task task : revokedActiveTasks) { | ||
| // 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 : revokedActiveTasks) { | ||
| if (!dirtyTasks.contains(task)) { | ||
| try { | ||
| task.postCommit(true); | ||
| } catch (final RuntimeException e) { | ||
| log.error("Exception caught while post-committing task " + task.id(), e); | ||
| firstException.compareAndSet(null, e); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| if (shouldCommitAdditionalTasks) { | ||
| for (final Task task : commitNeededActiveTasks) { | ||
| if (shouldCommitAdditionalTasks) { | ||
| for (final Task task : commitNeededActiveTasks) { | ||
| if (!dirtyTasks.contains(task)) { | ||
| try { | ||
| // for non-revoking active tasks, we should not enforce checkpoint | ||
| // since if it is EOS enabled, no checkpoint should be written while | ||
|
|
@@ -972,42 +1007,53 @@ void addRecordsToTasks(final ConsumerRecords<byte[], byte[]> records) { | |
| /** | ||
| * @throws TaskMigratedException if committing offsets failed (non-EOS) | ||
| * or if the task producer got fenced (EOS) | ||
| * @throws TimeoutException if task.timeout.ms has been exceeded (non-EOS) | ||
| * @throws TaskCorruptedException if committing offsets failed due to TimeoutException (EOS) | ||
| * @return number of committed offsets, or -1 if we are in the middle of a rebalance and cannot commit | ||
| */ | ||
| int commit(final Collection<Task> tasksToCommit) { | ||
| int committed = 0; | ||
| if (rebalanceInProgress) { | ||
| return -1; | ||
| committed = -1; | ||
| } else { | ||
| int committed = 0; | ||
| final Map<Task, Map<TopicPartition, OffsetAndMetadata>> consumedOffsetsAndMetadataPerTask = new HashMap<>(); | ||
| for (final Task task : tasksToCommit) { | ||
| if (task.commitNeeded()) { | ||
| final Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata = task.prepareCommit(); | ||
| if (task.isActive()) { | ||
| consumedOffsetsAndMetadataPerTask.put(task, offsetAndMetadata); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| try { | ||
| commitOffsetsOrTransaction(consumedOffsetsAndMetadataPerTask); | ||
|
|
||
| for (final Task task : tasksToCommit) { | ||
| if (task.commitNeeded()) { | ||
| task.clearTaskTimeout(); | ||
| ++committed; | ||
| task.postCommit(false); | ||
| } | ||
| } | ||
| committed = commitAndFillInConsumedOffsetsAndMetadataPerTaskMap(tasksToCommit, consumedOffsetsAndMetadataPerTask); | ||
| } catch (final TimeoutException timeoutException) { | ||
| consumedOffsetsAndMetadataPerTask | ||
| .keySet() | ||
| .forEach(t -> t.maybeInitTaskTimeoutOrThrow(time.milliseconds(), timeoutException)); | ||
| } | ||
| } | ||
| return committed; | ||
| } | ||
|
|
||
| /** | ||
| * @param consumedOffsetsAndMetadataPerTask an empty map that will be filled in with the prepared offsets | ||
| */ | ||
| private int commitAndFillInConsumedOffsetsAndMetadataPerTaskMap(final Collection<Task> tasksToCommit, | ||
|
Member
Author
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. Just pulled all the actual contents, excluding the TimeoutException + |
||
| final Map<Task, Map<TopicPartition, OffsetAndMetadata>> consumedOffsetsAndMetadataPerTask) { | ||
| int committed = 0; | ||
|
|
||
| return committed; | ||
| for (final Task task : tasksToCommit) { | ||
| if (task.commitNeeded()) { | ||
| final Map<TopicPartition, OffsetAndMetadata> offsetAndMetadata = task.prepareCommit(); | ||
| if (task.isActive()) { | ||
| consumedOffsetsAndMetadataPerTask.put(task, offsetAndMetadata); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| commitOffsetsOrTransaction(consumedOffsetsAndMetadataPerTask); | ||
|
|
||
| for (final Task task : tasksToCommit) { | ||
| if (task.commitNeeded()) { | ||
| task.clearTaskTimeout(); | ||
| ++committed; | ||
| task.postCommit(false); | ||
| } | ||
| } | ||
| return committed; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -1027,6 +1073,11 @@ int maybeCommitActiveTasksPerUserRequested() { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * @throws TaskMigratedException if committing offsets failed due to CommitFailedException (non-EOS) | ||
| * @throws TimeoutException if committing offsets failed due to TimeoutException (non-EOS) | ||
| * @throws TaskCorruptedException if committing offsets failed due to TimeoutException (EOS) | ||
| */ | ||
| private void commitOffsetsOrTransaction(final Map<Task, Map<TopicPartition, OffsetAndMetadata>> offsetsPerTask) { | ||
| log.debug("Committing task offsets {}", offsetsPerTask.entrySet().stream().collect(Collectors.toMap(t -> t.getKey().id(), Entry::getValue))); // avoid logging actual Task objects | ||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.