-
Notifications
You must be signed in to change notification settings - Fork 15.3k
KAFKA-20710: Share coordinator - Fence stale periodic jobs #22603
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
base: trunk
Are you sure you want to change the base?
Changes from all commits
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 |
|---|---|---|
|
|
@@ -76,6 +76,7 @@ | |
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.function.IntSupplier; | ||
|
|
||
| import static org.apache.kafka.coordinator.common.runtime.CoordinatorOperationExceptionHelper.handleOperationException; | ||
|
|
@@ -141,6 +142,8 @@ public class ShareCoordinatorService implements ShareCoordinator { | |
| */ | ||
| private volatile boolean shouldRunPeriodicJob; | ||
|
|
||
| private final AtomicLong periodicJobGeneration = new AtomicLong(); | ||
|
|
||
| public static class Builder { | ||
| private final int nodeId; | ||
| private final ShareCoordinatorConfig config; | ||
|
|
@@ -319,9 +322,9 @@ public void startup( | |
| log.info("Startup complete."); | ||
| } | ||
|
|
||
| private void setupPeriodicJobs() { | ||
| setupRecordPruning(); | ||
| setupSnapshotColdPartitions(); | ||
| private void setupPeriodicJobs(long generation) { | ||
| setupRecordPruning(generation); | ||
| setupSnapshotColdPartitions(generation); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -331,11 +334,15 @@ private void setupPeriodicJobs() { | |
| */ | ||
| // Visibility for tests | ||
| void setupRecordPruning() { | ||
| setupRecordPruning(periodicJobGeneration.get()); | ||
| } | ||
|
|
||
| private void setupRecordPruning(long generation) { | ||
| log.debug("Scheduling share-group state topic prune job."); | ||
| timer.add(new TimerTask(config.shareCoordinatorTopicPruneIntervalMs()) { | ||
| @Override | ||
| public void run() { | ||
| if (!shouldRunPeriodicJob) { | ||
| if (!shouldRunPeriodicJob(generation)) { | ||
| return; | ||
| } | ||
| List<CompletableFuture<Void>> futures = new ArrayList<>(); | ||
|
|
@@ -346,8 +353,11 @@ public void run() { | |
| if (exp != null) { | ||
| log.error("Received error in share-group state topic prune.", exp); | ||
| } | ||
| if (!shouldRunPeriodicJob(generation)) { | ||
| return; | ||
| } | ||
| // Perpetual recursion, failure or not. | ||
| setupRecordPruning(); | ||
| setupRecordPruning(generation); | ||
| }); | ||
| } | ||
| }); | ||
|
|
@@ -415,11 +425,15 @@ private CompletableFuture<Void> performRecordPruning(TopicPartition tp) { | |
| */ | ||
| // Visibility for tests | ||
| void setupSnapshotColdPartitions() { | ||
| setupSnapshotColdPartitions(periodicJobGeneration.get()); | ||
| } | ||
|
|
||
| private void setupSnapshotColdPartitions(long generation) { | ||
| log.debug("Scheduling cold share-partition snapshotting."); | ||
| timer.add(new TimerTask(config.shareCoordinatorColdPartitionSnapshotIntervalMs()) { | ||
| @Override | ||
| public void run() { | ||
| if (!shouldRunPeriodicJob) { | ||
| if (!shouldRunPeriodicJob(generation)) { | ||
| return; | ||
| } | ||
| List<CompletableFuture<Void>> futures = runtime.scheduleWriteAllOperation( | ||
|
|
@@ -432,7 +446,10 @@ public void run() { | |
| if (exp != null) { | ||
| log.error("Received error while snapshotting cold partitions.", exp); | ||
| } | ||
| setupSnapshotColdPartitions(); | ||
| if (!shouldRunPeriodicJob(generation)) { | ||
| return; | ||
| } | ||
| setupSnapshotColdPartitions(generation); | ||
| }); | ||
| } | ||
| }); | ||
|
|
@@ -1137,8 +1154,9 @@ public void onMetadataUpdate(MetadataDelta delta, MetadataImage newImage) { | |
| // 1 1 no op on flag, do not call jobs | ||
| if (enabled ^ shouldRunPeriodicJob) { | ||
| shouldRunPeriodicJob = enabled; | ||
| long generation = periodicJobGeneration.incrementAndGet(); | ||
| if (enabled) { | ||
| setupPeriodicJobs(); | ||
| setupPeriodicJobs(generation); | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -1182,4 +1200,8 @@ private boolean isShareGroupsEnabled(MetadataImage image) { | |
| boolean shouldRunPeriodicJob() { | ||
| return shouldRunPeriodicJob; | ||
| } | ||
|
|
||
| private boolean shouldRunPeriodicJob(long generation) { | ||
| return shouldRunPeriodicJob && periodicJobGeneration.get() == generation; | ||
|
Contributor
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 must not start work if it belongs to old generation. |
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -2066,6 +2066,82 @@ public void testPeriodicJobsDoNotRunWhenShareGroupsDisabled() throws Interrupted | |
| service.shutdown(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testPeriodicJobsDoNotDuplicateAfterDisableEnableWithInFlightJobs() throws InterruptedException { | ||
| CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime(); | ||
| PartitionWriter writer = mock(PartitionWriter.class); | ||
| MockTime time = new MockTime(); | ||
| MockTimer timer = spy(new MockTimer(time)); | ||
|
|
||
| Metrics metrics = new Metrics(); | ||
|
|
||
| ShareCoordinatorService service = spy(new ShareCoordinatorService( | ||
| new LogContext(), | ||
| ShareCoordinatorTestConfig.testConfig(), | ||
| runtime, | ||
| new ShareCoordinatorMetrics(metrics), | ||
| time, | ||
| timer, | ||
| writer | ||
| )); | ||
|
|
||
| CompletableFuture<Optional<Long>> firstPruneFuture = new CompletableFuture<>(); | ||
| CompletableFuture<Optional<Long>> secondPruneFuture = new CompletableFuture<>(); | ||
| CompletableFuture<Void> firstSnapshotFuture = new CompletableFuture<>(); | ||
| CompletableFuture<Void> secondSnapshotFuture = new CompletableFuture<>(); | ||
|
|
||
| when(runtime.<Optional<Long>>scheduleWriteOperation( | ||
| eq("write-state-record-prune"), | ||
| any(), | ||
| any() | ||
| )) | ||
| .thenReturn(firstPruneFuture) | ||
| .thenReturn(secondPruneFuture); | ||
|
|
||
| when(runtime.<Void>scheduleWriteAllOperation( | ||
| eq("snapshot-cold-partitions"), | ||
| any() | ||
| )) | ||
| .thenReturn(List.of(firstSnapshotFuture)) | ||
| .thenReturn(List.of(secondSnapshotFuture)); | ||
|
|
||
| service.startup(() -> 1); | ||
|
Contributor
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. two periodic jobs: |
||
|
|
||
| MetadataImage disabledImage = mock(MetadataImage.class, RETURNS_DEEP_STUBS); | ||
| when(disabledImage.features().finalizedVersions().getOrDefault(eq(ShareVersion.FEATURE_NAME), anyShort())).thenReturn((short) 0); | ||
|
|
||
| MetadataImage enabledImage = mockMetadataImageWithShareGroupsEnabled(); | ||
| service.onMetadataUpdate(mock(MetadataDelta.class), enabledImage); | ||
|
|
||
| verify(timer, times(2)).add(any()); | ||
| timer.advanceClock(30001L); | ||
|
Contributor
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. manually advances mock time so the scheduled timer tasks fire |
||
| verify(runtime, times(1)).scheduleWriteOperation( | ||
|
Contributor
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. both delayed tasks run once when we advance the timer |
||
| eq("write-state-record-prune"), | ||
| any(), | ||
| any() | ||
| ); | ||
| verify(runtime, times(1)).scheduleWriteAllOperation( | ||
| eq("snapshot-cold-partitions"), | ||
| any() | ||
| ); | ||
|
|
||
| service.onMetadataUpdate(mock(MetadataDelta.class), disabledImage); | ||
| assertFalse(service.shouldRunPeriodicJob()); | ||
|
|
||
| service.onMetadataUpdate(mock(MetadataDelta.class), enabledImage); | ||
| assertTrue(service.shouldRunPeriodicJob()); | ||
| verify(timer, times(4)).add(any()); | ||
|
Contributor
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. Without the fix, this became 6 because old prune and old snapshot completions each rescheduled one stale job.
Contributor
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. 2(old) + 4(new) |
||
|
|
||
| firstPruneFuture.complete(Optional.empty()); | ||
| firstSnapshotFuture.complete(null); | ||
|
|
||
| verify(timer, times(4)).add(any()); | ||
|
|
||
| checkMetrics(metrics); | ||
|
|
||
| service.shutdown(); | ||
| } | ||
|
|
||
| @Test | ||
| public void testShareStateTopicConfigs() { | ||
| CoordinatorRuntime<ShareCoordinatorShard, CoordinatorRecord> runtime = mockRuntime(); | ||
|
|
||
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.
protects duplicate timer chains - similar to epoch concept.