From e552869e59070aff9b05f5cd7406f2ad7c6d23a8 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 16 Mar 2026 19:09:47 -0700 Subject: [PATCH 01/27] Update reset endpoint to return a map of skipped offsets for each partition --- .../supervisor/RabbitStreamSupervisor.java | 6 +- .../kafka/supervisor/KafkaSupervisor.java | 12 +-- .../kinesis/supervisor/KinesisSupervisor.java | 4 +- .../supervisor/SupervisorManager.java | 76 ++++++++++++++++++- .../supervisor/SupervisorResource.java | 38 +++++++++- .../SeekableStreamIndexTaskRunner.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 65 ++++++++++++++-- .../supervisor/SupervisorResourceTest.java | 4 +- .../SeekableStreamSupervisorStateTest.java | 6 +- .../SeekableStreamSupervisorTestBase.java | 4 +- 10 files changed, 190 insertions(+), 27 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java index 9be4a77a8d4e..32ac2b9c8651 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java @@ -321,7 +321,7 @@ protected Map getTimeLagPerPartition(Map currentOffs } @Override - protected RabbitStreamDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map map) + public RabbitStreamDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map map) { return new RabbitStreamDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, map)); } @@ -374,7 +374,7 @@ public LagStats computeLagStats() } @Override - protected void updatePartitionLagFromStream() + public void updatePartitionLagFromStream() { getRecordSupplierLock().lock(); @@ -401,7 +401,7 @@ protected void updatePartitionLagFromStream() } @Override - protected Map getLatestSequencesFromStream() + public Map getLatestSequencesFromStream() { return latestSequenceFromStream != null ? latestSequenceFromStream : new HashMap<>(); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index e226db376764..e2062ef75aa1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -345,7 +345,7 @@ protected Map getTimeLagPerPartition(Map map) + public KafkaDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map map) { return new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, map)); } @@ -502,7 +502,7 @@ private Map getTimestampPerPartitionAtCurrentOffset(S * {@link #getPartitionRecordLag}. */ @Override - protected void updatePartitionLagFromStream() + public void updatePartitionLagFromStream() { if (getIoConfig().isEmitTimeLagMetrics()) { updatePartitionTimeAndRecordLagFromStream(); @@ -539,7 +539,7 @@ protected void updatePartitionLagFromStream() } @Override - protected Map getLatestSequencesFromStream() + public Map getLatestSequencesFromStream() { return latestSequenceFromStream != null ? latestSequenceFromStream : new HashMap<>(); } @@ -572,17 +572,17 @@ protected boolean isMultiTopic() * Gets the offsets as stored in the metadata store. The map returned will only contain * offsets from topic partitions that match the current supervisor config stream. This * override is needed because in the case of multi-topic, a user could have updated the supervisor - * config from single topic to mult-topic, where the new multi-topic pattern regex matches the + * config from single topic to multi-topic, where the new multi-topic pattern regex matches the * old config single topic. Without this override, the previously stored metadata for the single * topic would be deemed as different from the currently configure stream, and not be included in * the offset map returned. This implementation handles these cases appropriately. * - * @return the previoulsy stored offsets from metadata storage, possibly updated with offsets removed + * @return the previously stored offsets from metadata storage, possibly updated with offsets removed * for topics that do not match the currently configured supervisor topic. Topic partition keys may also be * updated to single topic or multi-topic depending on the supervisor config, as needed. */ @Override - protected Map getOffsetsFromMetadataStorage() + public Map getOffsetsFromMetadataStorage() { final DataSourceMetadata dataSourceMetadata = retrieveDataSourceMetadata(); if (checkSourceMetadataMatch(dataSourceMetadata)) { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 08491caa8ff5..9b434da54180 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -317,7 +317,7 @@ protected Map getTimeLagPerPartition(Map currentOf } @Override - protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( + public SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( String stream, Map map ) @@ -332,7 +332,7 @@ protected OrderedSequenceNumber makeSequenceNumber(String seq, boolean i } @Override - protected void updatePartitionLagFromStream() + public void updatePartitionLagFromStream() { KinesisRecordSupplier supplier = (KinesisRecordSupplier) recordSupplier; // this recordSupplier method is thread safe, so does not need to acquire the recordSupplierLock diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index d598fdd0910b..73bbe69a9407 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import org.apache.druid.common.guava.FutureUtils; @@ -37,7 +38,10 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -273,7 +277,7 @@ public void stop() Preconditions.checkState(started, "SupervisorManager not started"); List> stopFutures = new ArrayList<>(); synchronized (lock) { - log.info("Stopping [%d] supervisors", supervisors.keySet().size()); + log.info("Stopping [%d] supervisors", supervisors.size()); for (String id : supervisors.keySet()) { try { stopFutures.add(supervisors.get(id).lhs.stopAsync()); @@ -365,6 +369,76 @@ public boolean resetSupervisor(String id, @Nullable DataSourceMetadata resetData return true; } + /** + * Resets a supervisor to latest offsets and returns the skipped offset ranges. + * Requirements: + * - Supervisor must be RUNNING - needs active stream connection + * - Supervisor must be a SeekableStreamSupervisor (Kafka, Kinesis) + * - useEarliestOffset must be false (otherwise supervisor always starts from earliest) + * @param id supervisor ID + * @return Map containing supervisorId and skipped offset ranges + * @throws IllegalArgumentException if supervisor doesn't exist or if useEarliestOffset is true + * @throws IllegalStateException if supervisor is not running + */ + public Map resetSupervisorAndReturnSkippedOffsets(String id) + { + Preconditions.checkState(started, "SupervisorManager not started"); + Preconditions.checkNotNull(id, "id"); + + Pair supervisorPair = supervisors.get(id); + if (supervisorPair == null || supervisorPair.lhs == null || supervisorPair.rhs == null) { + throw new IAE("Supervisor[%s] does not exist", id); + } + if (!(supervisorPair.lhs instanceof SeekableStreamSupervisor)) { + throw new IAE("Supervisor[%s] is not a SeekableStreamSupervisor", id); + } + SeekableStreamSupervisor streamSupervisor = (SeekableStreamSupervisor) supervisorPair.lhs; + + // Verify useEarliestOffset is false + if (streamSupervisor.getIoConfig().isUseEarliestSequenceNumber()) { + throw new IAE("Reset with skipped offsets is not supported when useEarliestOffset is true."); + } + + // We need an active recordSupplier to query the latest offsets from the stream + if (supervisorPair.lhs.getState() != SupervisorStateManager.BasicState.RUNNING) { + throw new ISE("A running supervisor is required to query the latest offsets from the stream"); + } + + log.info("Capturing latest offsets from stream for supervisor[%s]", id); + streamSupervisor.updatePartitionLagFromStream(); + Map latestOffsets = streamSupervisor.getLatestSequencesFromStream(); + + log.info("Capturing checkpointed offsets for supervisor[%s]", id); + Map startOffsets = streamSupervisor.getOffsetsFromMetadataStorage(); + + log.info("Resetting supervisor[%s] metadata to latest offsets", id); + DataSourceMetadata resetMetadata = streamSupervisor.createDataSourceMetaDataForReset( + streamSupervisor.getIoConfig().getStream(), + latestOffsets + ); + + streamSupervisor.resetOffsets(resetMetadata); + + // Reset autoscaler if present + SupervisorTaskAutoScaler autoscaler = autoscalers.get(id); + if (autoscaler != null) { + autoscaler.reset(); + } + + Map skippedRanges = streamSupervisor.calculateSkippedOffsetRanges(startOffsets, latestOffsets); + + log.info( + "Successfully reset supervisor[%s] to latest. Skipped ranges: %s", + id, + skippedRanges + ); + + return ImmutableMap.of( + "id", id, + "skippedOffsets", skippedRanges + ); + } + public boolean checkPointDataSourceMetadata( String supervisorId, int taskGroupId, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index 6b145be07e4a..b47faa886e18 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -592,9 +592,16 @@ public Response specGetHistory( @Path("/{id}/reset") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(SupervisorResourceFilter.class) - public Response reset(@PathParam("id") final String id) + public Response reset( + @PathParam("id") final String id, + @QueryParam("returnSkippedOffsets") Boolean returnSkippedOffsets + ) { - return handleResetRequest(id, null); + if (Boolean.TRUE.equals(returnSkippedOffsets)) { + return handleResetAndReturnSkippedOffsets(id); + } else { + return handleResetRequest(id, null); + } } @POST @@ -628,6 +635,33 @@ private Response handleResetRequest( ); } + private Response handleResetAndReturnSkippedOffsets(final String id) + { + return asLeaderWithSupervisorManager( + manager -> { + try { + Map result = manager.resetSupervisorAndReturnSkippedOffsets(id); + return Response.ok(result).build(); + } + catch (IllegalArgumentException e) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of("error", e.getMessage())) + .build(); + } + catch (IllegalStateException e) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of("error", e.getMessage())) + .build(); + } + catch (Exception e) { + return Response.serverError() + .entity(ImmutableMap.of("error", e.getMessage())) + .build(); + } + } + ); + } + private Response asLeaderWithSupervisorManager(Function f) { Optional supervisorManager = taskMaster.getSupervisorManager(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index a448e01f9189..1ca091771286 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -821,7 +821,7 @@ public void onFailure(Throwable t) caughtExceptionInner = e; if (Throwables.getRootCause(e) instanceof InterruptedException) { // Suppress InterruptedException stack trace to avoid flooding the logs - log.error("Encounted InterrupedException in run() before persisting"); + log.error("Encountered InterrupedException in run() before persisting"); } else { log.error(e, "Encountered exception in run() before persisting."); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index d257c1e4e8cb..5238082ae9be 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1916,6 +1916,51 @@ public void gracefulShutdownInternal() throws ExecutionException, InterruptedExc checkTaskDuration(); } + /** + * Calculates the skipped offset ranges between start and end offsets. + * Returns a map with partition ID as key and [startOffset, endOffset] array as value. + * + * @param startOffsets Starting offsets (last checkpointed) + * @param endOffsets Ending offsets (latest from stream) + * @return Map of partition ID (as String) to offset range [start, end] + */ + public Map calculateSkippedOffsetRanges( + Map startOffsets, + Map endOffsets + ) + { + Map skippedRanges = new HashMap<>(); + + for (Entry entry : endOffsets.entrySet()) { + PartitionIdType partition = entry.getKey(); + SequenceOffsetType endOffset = entry.getValue(); + SequenceOffsetType startOffset = (startOffsets != null) ? startOffsets.get(partition) : null; + + if (startOffset != null) { + // Both start and end exist - calculate range + skippedRanges.put( + partition.toString(), + ImmutableMap.of( + "start", startOffset, + "end", endOffset + ) + ); + } else { + // No checkpoint exists for this partition + skippedRanges.put( + partition.toString(), + ImmutableMap.of( + "start", "none", + "end", endOffset, + "note", "No committed offset found for this partition" + ) + ); + } + } + + return skippedRanges; + } + @VisibleForTesting public void resetInternal(DataSourceMetadata dataSourceMetadata) { @@ -3144,9 +3189,13 @@ private boolean updatePartitionDataFromStream() } /** - * gets mapping of partitions in stream to their latest offsets. + * Gets mapping of partitions in stream to their latest offsets. + * This returns the cached latest offsets that were captured by the most recent + * call to {@link #updatePartitionLagFromStream()}. + * + * @return Map of partition ID to latest sequence offset */ - protected Map getLatestSequencesFromStream() + public Map getLatestSequencesFromStream() { return new HashMap<>(); } @@ -4263,7 +4312,13 @@ private OrderedSequenceNumber getOffsetFromStorageForPartiti } } - protected Map getOffsetsFromMetadataStorage() + /** + * Gets the last committed offsets from metadata storage. + * This represents the offsets that have been successfully published by tasks. + * + * @return Map of partition ID to sequence offset, or empty map if no metadata exists or stream doesn't match + */ + public Map getOffsetsFromMetadataStorage() { final DataSourceMetadata dataSourceMetadata = retrieveDataSourceMetadata(); if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata @@ -4478,7 +4533,7 @@ private void updateCurrentOffsets() throws InterruptedException, ExecutionExcept coalesceAndAwait(futures); } - protected abstract void updatePartitionLagFromStream(); + public abstract void updatePartitionLagFromStream(); /** * Gets 'lag' of currently processed offset behind latest offset as a measure of difference between offsets. @@ -4716,7 +4771,7 @@ protected abstract List sequence * @return specific instance of datasource metadata */ - protected abstract SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( + public abstract SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( String stream, Map map ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 6c099a53b3a1..f7b9a17f6e4e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -1257,12 +1257,12 @@ public void testReset() )).andReturn(false); replayAll(); - Response response = supervisorResource.reset("my-id"); + Response response = supervisorResource.reset("my-id", false); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(ImmutableMap.of("id", "my-id"), response.getEntity()); - response = supervisorResource.reset("my-id-2"); + response = supervisorResource.reset("my-id-2", false); Assert.assertEquals(404, response.getStatus()); Assert.assertEquals("my-id", id1.getValue()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 3f534de8068e..301d5d174ba0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -3074,7 +3074,7 @@ protected String baseTaskName() } @Override - protected void updatePartitionLagFromStream() + public void updatePartitionLagFromStream() { // do nothing } @@ -3170,7 +3170,7 @@ protected boolean doesTaskMatchSupervisor(Task task) } @Override - protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( + public SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( String stream, Map map ) @@ -3286,7 +3286,7 @@ public LagStats computeLagStats() } @Override - protected Map getLatestSequencesFromStream() + public Map getLatestSequencesFromStream() { return streamOffsets; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java index 35d063e88e1e..4746fef689f4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java @@ -123,7 +123,7 @@ protected String baseTaskName() } @Override - protected void updatePartitionLagFromStream() + public void updatePartitionLagFromStream() { // do nothing } @@ -203,7 +203,7 @@ protected boolean doesTaskMatchSupervisor(Task task) } @Override - protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( + public SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( String stream, Map map ) From 0c13ba88af51a774e137e5d3d33c996147bc50d6 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 16 Mar 2026 19:16:37 -0700 Subject: [PATCH 02/27] Checkstyle fixes --- .../indexing/overlord/supervisor/SupervisorManager.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 73bbe69a9407..048dd33bd300 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -41,7 +41,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -427,11 +426,7 @@ public Map resetSupervisorAndReturnSkippedOffsets(String id) Map skippedRanges = streamSupervisor.calculateSkippedOffsetRanges(startOffsets, latestOffsets); - log.info( - "Successfully reset supervisor[%s] to latest. Skipped ranges: %s", - id, - skippedRanges - ); + log.info("Successfully reset supervisor[%s] to latest. Skipped ranges: %s", id, skippedRanges); return ImmutableMap.of( "id", id, From d5da02ce7f91fc70bf225b70e4c80b6898da8132 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 16 Mar 2026 21:00:43 -0700 Subject: [PATCH 03/27] Formatting fix for response --- .../overlord/supervisor/SupervisorManager.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 048dd33bd300..023b7171e748 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -424,7 +424,7 @@ public Map resetSupervisorAndReturnSkippedOffsets(String id) autoscaler.reset(); } - Map skippedRanges = streamSupervisor.calculateSkippedOffsetRanges(startOffsets, latestOffsets); + Map skippedRanges = streamSupervisor.calculateSkippedOffsetRanges(startOffsets, latestOffsets); log.info("Successfully reset supervisor[%s] to latest. Skipped ranges: %s", id, skippedRanges); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 5238082ae9be..4f2aa090f6f5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1922,14 +1922,14 @@ public void gracefulShutdownInternal() throws ExecutionException, InterruptedExc * * @param startOffsets Starting offsets (last checkpointed) * @param endOffsets Ending offsets (latest from stream) - * @return Map of partition ID (as String) to offset range [start, end] + * @return Map of partition ID to offset range [start, end] */ - public Map calculateSkippedOffsetRanges( + public Map calculateSkippedOffsetRanges( Map startOffsets, Map endOffsets ) { - Map skippedRanges = new HashMap<>(); + Map skippedRanges = new HashMap<>(); for (Entry entry : endOffsets.entrySet()) { PartitionIdType partition = entry.getKey(); @@ -1939,7 +1939,7 @@ public Map calculateSkippedOffsetRanges( if (startOffset != null) { // Both start and end exist - calculate range skippedRanges.put( - partition.toString(), + partition, ImmutableMap.of( "start", startOffset, "end", endOffset @@ -1948,7 +1948,7 @@ public Map calculateSkippedOffsetRanges( } else { // No checkpoint exists for this partition skippedRanges.put( - partition.toString(), + partition, ImmutableMap.of( "start", "none", "end", endOffset, From 4034a987cadc6e2fff23c03dbe759c6ada8b0421 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 18 Mar 2026 12:49:11 -0700 Subject: [PATCH 04/27] Support unsupervised task for SeekableStreamIndexTask --- .../RabbitStreamIndexTaskIOConfig.java | 6 +- .../supervisor/RabbitStreamSupervisor.java | 3 +- .../kafka/KafkaIndexTaskIOConfig.java | 9 +- .../kafka/supervisor/KafkaSupervisor.java | 3 +- .../kinesis/KinesisIndexTaskIOConfig.java | 9 +- .../SeekableStreamIndexTaskIOConfig.java | 12 ++- .../SeekableStreamIndexTaskRunner.java | 100 +++++++++++++++--- .../supervisor/SupervisorResourceTest.java | 37 +++++++ .../TestSeekableStreamIndexTaskIOConfig.java | 3 +- .../SeekableStreamSupervisorStateTest.java | 6 +- .../SeekableStreamSupervisorTestBase.java | 3 +- .../org/apache/druid/cli/CliPeonTest.java | 3 +- 12 files changed, 163 insertions(+), 31 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java index 721e66f6f3af..30780155ce67 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java @@ -54,7 +54,8 @@ public RabbitStreamIndexTaskIOConfig( @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("uri") String uri, - @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes + @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes, + @JsonProperty("supervised") @Nullable Boolean supervised ) { super( @@ -66,7 +67,8 @@ public RabbitStreamIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + supervised ); this.pollTimeout = pollTimeout != null ? pollTimeout : RabbitStreamSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS; diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java index 32ac2b9c8651..f5241c1300c0 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java @@ -203,7 +203,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( maximumMessageTime, ioConfig.getInputFormat(), rabbitConfig.getUri(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + true ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java index 07c0f80fbe83..d70864d1f22b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java @@ -64,7 +64,8 @@ public KafkaIndexTaskIOConfig( @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("configOverrides") @Nullable KafkaConfigOverrides configOverrides, @JsonProperty("multiTopic") @Nullable Boolean multiTopic, - @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes + @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes, + @JsonProperty("supervised") @Nullable Boolean supervised ) { super( @@ -78,7 +79,8 @@ public KafkaIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + supervised ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); @@ -128,7 +130,8 @@ public KafkaIndexTaskIOConfig( inputFormat, configOverrides, KafkaSupervisorIOConfig.DEFAULT_IS_MULTI_TOPIC, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + true ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index e2062ef75aa1..3da1932b2d3e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -223,7 +223,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ioConfig.getInputFormat(), kafkaIoConfig.getConfigOverrides(), kafkaIoConfig.isMultiTopic(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + true ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java index 2df59bbca35f..0044e7cfb4a0 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java @@ -79,7 +79,8 @@ public KinesisIndexTaskIOConfig( @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, - @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes + @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes, + @JsonProperty("supervised") @Nullable Boolean supervised ) { super( @@ -91,7 +92,8 @@ public KinesisIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + supervised ); Preconditions.checkArgument( getEndSequenceNumbers().getPartitionSequenceNumberMap() @@ -139,7 +141,8 @@ public KinesisIndexTaskIOConfig( fetchDelayMillis, awsAssumedRoleArn, awsExternalId, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + true ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java index 8f92f76e8360..7681bb78fc2b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java @@ -32,6 +32,7 @@ public abstract class SeekableStreamIndexTaskIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; + private static final boolean DEFAULT_SUPERVISED = true; private final Integer taskGroupId; private final String baseSequenceName; @@ -42,6 +43,7 @@ public abstract class SeekableStreamIndexTaskIOConfig( - stream, - sequenceToCheckpoint.getStartOffsets(), - sequenceToCheckpoint.getExclusiveStartPartitions() - ) - ) - ); - if (!toolbox.getTaskActionClient().submit(checkpointAction)) { - throw new ISE("Checkpoint request with sequences [%s] failed, dying", currOffsets); + + if (ioConfig.isSupervised()) { + // Normal checkpoint with supervisor coordination + requestPause(); + final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction( + getSupervisorId(), + ioConfig.getTaskGroupId(), + null, + createDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>( + stream, + sequenceToCheckpoint.getStartOffsets(), + sequenceToCheckpoint.getExclusiveStartPartitions() + ) + ) + ); + if (!toolbox.getTaskActionClient().submit(checkpointAction)) { + throw new ISE("Checkpoint request with sequences [%s] failed, dying", currOffsets); + } + } else { + // Unsupervised task: skip supervisor coordination but still create a new sequence to respect maxRowsPerSegment + log.info( + "Task is unsupervised - skipping checkpoint coordination for sequence [%s], but finalizing sequence to respect segment size limits. Current offsets: [%s]", + sequenceToCheckpoint.getSequenceName(), + currOffsets + ); + + // Set end offsets for the current sequence to current position to trigger segment publish + sequenceToCheckpoint.setEndOffsets(currOffsets); + + // Determine exclusive start partitions for the new sequence + final Set exclusiveStartPartitions; + if (isEndOffsetExclusive()) { + // When end offsets are exclusive, no partitions need exclusive start + exclusiveStartPartitions = Collections.emptySet(); + } else { + // When end offsets are inclusive, all partitions in the new sequence need exclusive start + exclusiveStartPartitions = ImmutableSet.copyOf(currOffsets.keySet()); + } + + // Create a new sequence starting from current offsets, ending at the task's overall end offsets + final SequenceMetadata newSequence = createNewSequence( + nextSequenceNumber(), + currOffsets, + this.endOffsets, // Continue to the task's end offsets + exclusiveStartPartitions + ); + sequences.add(newSequence); + + log.info( + "Created new sequence [%s] starting from offsets [%s], will read until end offsets [%s]", + newSequence.getSequenceName(), + currOffsets, + this.endOffsets + ); } } } @@ -1311,6 +1351,36 @@ public SequenceMetadata getLastSequenceMeta return sequences.get(sequences.size() - 1); } + /** + * Returns the next sequence ID to use when creating a new sequence. + */ + private int nextSequenceNumber() + { + return getLastSequenceMetadata().getSequenceId() + 1; + } + + /** + * Creates a new SequenceMetadata for intermediate checkpoints when the task is unsupervised. + * This allows respecting maxRowsPerSegment without requiring supervisor coordination. + */ + private SequenceMetadata createNewSequence( + int sequenceId, + Map startOffsets, + Map endOffsets, + Set exclusiveStartPartitions + ) + { + return new SequenceMetadata<>( + sequenceId, + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), sequenceId), + startOffsets, + endOffsets, + false, // not checkpointed + exclusiveStartPartitions, + getTaskLockType() + ); + } + /** * Returns true if the given record has already been read, based on lastReadOffsets. */ diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index f7b9a17f6e4e..584d52749fcf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -1327,6 +1327,43 @@ public void testResetOffsets() verifyAll(); } + @Test + public void testResetOffsetsAndReturnSkippedOffsets() + { + Capture id1 = Capture.newInstance(); + Map expectedResult = ImmutableMap.of( + "id", "my-id", + "skippedOffsets", ImmutableMap.of( + "0", ImmutableMap.of("start", 50L, "end", 100L), + "1", ImmutableMap.of("start", 150L, "end", 200L), + "2", ImmutableMap.of("start", 250L, "end", 300L) + ) + ); + + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); + EasyMock.expect(supervisorManager.resetSupervisorAndReturnSkippedOffsets( + EasyMock.capture(id1) + )).andReturn(expectedResult); + replayAll(); + + Response response = supervisorResource.reset("my-id", true); + + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(expectedResult, response.getEntity()); + Assert.assertEquals("my-id", id1.getValue()); + verifyAll(); + + resetAll(); + + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()); + replayAll(); + + response = supervisorResource.reset("my-id", true); + + Assert.assertEquals(503, response.getStatus()); + verifyAll(); + } + @Test public void testNoopSupervisorSpecSerde() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TestSeekableStreamIndexTaskIOConfig.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TestSeekableStreamIndexTaskIOConfig.java index 1fa7993b69a6..6e663e26308c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TestSeekableStreamIndexTaskIOConfig.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TestSeekableStreamIndexTaskIOConfig.java @@ -38,7 +38,8 @@ public TestSeekableStreamIndexTaskIOConfig() DateTimes.nowUtc().minusDays(2), DateTimes.nowUtc(), new CsvInputFormat(null, null, true, null, 0, null), - Duration.standardHours(2).getStandardMinutes() + Duration.standardHours(2).getStandardMinutes(), + true ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 301d5d174ba0..2fb102313bcf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -3114,7 +3114,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( minimumMessageTime, maximumMessageTime, ioConfig.getInputFormat(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + true ) { }; @@ -3479,7 +3480,8 @@ private static SeekableStreamIndexTaskIOConfig createTaskIoConfigExt( minimumMessageTime, maximumMessageTime, ioConfig.getInputFormat(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + true ) { }; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java index 4746fef689f4..1e3bb7bdc5b9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java @@ -163,7 +163,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( minimumMessageTime, maximumMessageTime, ioConfig.getInputFormat(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + true ) { }; diff --git a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java index de3812a33eea..3849740108e5 100644 --- a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java +++ b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java @@ -478,7 +478,8 @@ public TestSeekableStreamIndexTaskIOConfig() DateTimes.nowUtc().minusDays(2), DateTimes.nowUtc(), new CsvInputFormat(null, null, true, null, 0, null), - Duration.standardHours(2).getStandardMinutes() + Duration.standardHours(2).getStandardMinutes(), + true ); } } From 89fc7a99312eec340b10acc854c14dfc6d0c3aac Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 18 Mar 2026 14:24:06 -0700 Subject: [PATCH 05/27] Minor refactor --- .../SeekableStreamIndexTaskRunner.java | 40 ++++--------------- 1 file changed, 7 insertions(+), 33 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index b525786a6b29..a0fe823bf1b8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -838,11 +838,15 @@ public void onFailure(Throwable t) } // Create a new sequence starting from current offsets, ending at the task's overall end offsets - final SequenceMetadata newSequence = createNewSequence( - nextSequenceNumber(), + final int nextSequenceId = getLastSequenceMetadata().getSequenceId() + 1; + final SequenceMetadata newSequence = new SequenceMetadata<>( + nextSequenceId, + StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), nextSequenceId), currOffsets, this.endOffsets, // Continue to the task's end offsets - exclusiveStartPartitions + false, // not checkpointed + exclusiveStartPartitions, + getTaskLockType() ); sequences.add(newSequence); @@ -1351,36 +1355,6 @@ public SequenceMetadata getLastSequenceMeta return sequences.get(sequences.size() - 1); } - /** - * Returns the next sequence ID to use when creating a new sequence. - */ - private int nextSequenceNumber() - { - return getLastSequenceMetadata().getSequenceId() + 1; - } - - /** - * Creates a new SequenceMetadata for intermediate checkpoints when the task is unsupervised. - * This allows respecting maxRowsPerSegment without requiring supervisor coordination. - */ - private SequenceMetadata createNewSequence( - int sequenceId, - Map startOffsets, - Map endOffsets, - Set exclusiveStartPartitions - ) - { - return new SequenceMetadata<>( - sequenceId, - StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), sequenceId), - startOffsets, - endOffsets, - false, // not checkpointed - exclusiveStartPartitions, - getTaskLockType() - ); - } - /** * Returns true if the given record has already been read, based on lastReadOffsets. */ From 84866f4209c2c7b8621f2ca9f198419bf17cb018 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 18 Mar 2026 20:38:58 -0700 Subject: [PATCH 06/27] Use addSequence helper method --- .../indexing/seekablestream/SeekableStreamIndexTaskRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index a0fe823bf1b8..a139a1e1d7f6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -848,7 +848,7 @@ public void onFailure(Throwable t) exclusiveStartPartitions, getTaskLockType() ); - sequences.add(newSequence); + addSequence(newSequence); log.info( "Created new sequence [%s] starting from offsets [%s], will read until end offsets [%s]", From 85ce0223fbded2a6fb73782aae2252176a1e65d5 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Thu, 19 Mar 2026 15:59:28 -0700 Subject: [PATCH 07/27] Automate backfill task submission for KafkaSupervisor --- .../kafka/supervisor/KafkaSupervisor.java | 80 +++++++++++++++++++ .../supervisor/SupervisorManager.java | 2 + .../supervisor/SeekableStreamSupervisor.java | 23 ++++++ 3 files changed, 105 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 3da1932b2d3e..997a046c32ba 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Objects; +import com.google.common.base.Optional; import com.google.common.collect.Sets; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.data.input.kafka.KafkaRecordEntity; @@ -41,6 +42,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; @@ -262,6 +264,84 @@ protected List startOffsets, + Map endOffsets + ) + { + if (startOffsets == null || startOffsets.isEmpty() || endOffsets == null || endOffsets.isEmpty()) { + log.info("No offsets to backfill, skipping backfill task submission"); + return; + } + + try { + String backfillSupervisorId = spec.getDataSchema().getDataSource() + "_backfill"; + String baseSequenceName = generateSequenceName( + startOffsets, + null, // minimumMessageTime - process all data in range + null, // maximumMessageTime - process all data in range + spec.getDataSchema(), + spec.getTuningConfig() + ); + + KafkaSupervisorIOConfig kafkaIoConfig = spec.getIoConfig(); + KafkaIndexTaskIOConfig backfillIoConfig = new KafkaIndexTaskIOConfig( + 0, + baseSequenceName, + null, + null, + new SeekableStreamStartSequenceNumbers<>(kafkaIoConfig.getStream(), startOffsets, Collections.emptySet()), + new SeekableStreamEndSequenceNumbers<>(kafkaIoConfig.getStream(), endOffsets), + kafkaIoConfig.getConsumerProperties(), + kafkaIoConfig.getPollTimeout(), + false, // useTransaction = false for backfill (no supervisor coordination) + null, // minimumMessageTime - no time filtering for backfill + null, // maximumMessageTime - no time filtering for backfill + kafkaIoConfig.getInputFormat(), + kafkaIoConfig.getConfigOverrides(), + kafkaIoConfig.isMultiTopic(), + null, // refreshRejectionPeriodsInMinutes - don't refresh rejection periods for backfill + false // supervised = false + ); + + // Create backfill task with different supervisorId + String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName); + Map context = createBaseTaskContexts(); + // Use APPEND locks to allow writing to intervals that may overlap with main supervisor + context.put("useConcurrentLocks", true); + + KafkaIndexTask backfillTask = new KafkaIndexTask( + taskId, + backfillSupervisorId, // Use backfill supervisorId instead of spec.getId() + new TaskResource(baseSequenceName, 1), + spec.getDataSchema(), + spec.getTuningConfig(), + backfillIoConfig, + context, + sortingMapper, + null // no server priority for backfill tasks + ); + + Optional taskQueue = getTaskMaster().getTaskQueue(); + if (taskQueue.isPresent()) { + log.info( + "Submitting backfill task[%s] with supervisorId[%s] for offsets from [%s] to [%s]", + taskId, + backfillSupervisorId, + startOffsets, + endOffsets + ); + taskQueue.get().add(backfillTask); + } else { + log.error("Failed to submit backfill task because I'm not the leader!"); + } + } + catch (Exception e) { + log.error(e, "Failed to submit backfill task, skipping backfill"); + } + } + @Override protected Map getPartitionRecordLag() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 023b7171e748..19a3d9c231d4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -426,6 +426,8 @@ public Map resetSupervisorAndReturnSkippedOffsets(String id) Map skippedRanges = streamSupervisor.calculateSkippedOffsetRanges(startOffsets, latestOffsets); + streamSupervisor.submitBackfillTask(startOffsets, latestOffsets); + log.info("Successfully reset supervisor[%s] to latest. Skipped ranges: %s", id, skippedRanges); return ImmutableMap.of( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 4f2aa090f6f5..32256b16eef2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -4621,6 +4621,11 @@ public SeekableStreamSupervisorIOConfig getIoConfig() return ioConfig; } + protected TaskMaster getTaskMaster() + { + return taskMaster; + } + @Override public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) { @@ -4764,6 +4769,24 @@ protected abstract List startOffsets, + Map endOffsets + ) + { + log.info("submitBackfillTask not implemented for supervisor[%s], skipping backfill submission", supervisorId); + } + /** * creates a specific instance of kafka/kinesis datasource metadata. Only used for reset. * From 543b9c719ff9528472db47484cd041e28e8aeb9d Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Thu, 19 Mar 2026 16:17:15 -0700 Subject: [PATCH 08/27] Switch from java.util.Optional to com.google.common.base.Optional --- .../druid/indexing/kafka/supervisor/KafkaSupervisor.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 997a046c32ba..cf4155554bf1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -73,7 +73,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Optional; import java.util.Set; import java.util.TreeMap; import java.util.regex.Pattern; @@ -391,7 +390,7 @@ private Map getRecordLagPerPartitionInLatestSequences Collectors.toMap( Entry::getKey, e -> e.getValue() != null - ? e.getValue() - Optional.ofNullable(currentOffsets.get(e.getKey())).orElse(0L) + ? e.getValue() - Optional.fromNullable(currentOffsets.get(e.getKey())).or(0L) : 0 ) ); From fa4b7a0d17564cc989ff4a66febfed026cd05676 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Thu, 19 Mar 2026 17:20:44 -0700 Subject: [PATCH 09/27] Support for configurable number of backfill tasks --- .../RabbitStreamSupervisorIOConfig.java | 3 +- .../kafka/supervisor/KafkaSupervisor.java | 144 +++++++++++------- .../supervisor/KafkaSupervisorIOConfig.java | 6 +- .../supervisor/KafkaIOConfigBuilder.java | 1 + .../KafkaSupervisorIOConfigTest.java | 3 + .../kafka/supervisor/KafkaSupervisorTest.java | 8 +- .../supervisor/KinesisSupervisorIOConfig.java | 3 +- .../SeekableStreamSupervisorIOConfig.java | 13 +- .../SeekableStreamSamplerSpecTest.java | 1 + .../SeekableStreamSupervisorIOConfigTest.java | 13 +- .../SeekableStreamSupervisorSpecTest.java | 4 + .../SeekableStreamSupervisorStateTest.java | 10 +- .../SeekableStreamSupervisorTestBase.java | 1 + 13 files changed, 150 insertions(+), 60 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java index 8aad5b762219..51a4c340658d 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java @@ -86,7 +86,8 @@ public RabbitStreamSupervisorIOConfig( lateMessageRejectionStartDateTime, new IdleConfig(null, null), stopTaskCount, - serverPriorityToReplicas + serverPriorityToReplicas, + null ); this.consumerProperties = consumerProperties; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index cf4155554bf1..61d63b683977 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -276,64 +276,106 @@ public void submitBackfillTask( try { String backfillSupervisorId = spec.getDataSchema().getDataSource() + "_backfill"; - String baseSequenceName = generateSequenceName( - startOffsets, - null, // minimumMessageTime - process all data in range - null, // maximumMessageTime - process all data in range - spec.getDataSchema(), - spec.getTuningConfig() - ); - KafkaSupervisorIOConfig kafkaIoConfig = spec.getIoConfig(); - KafkaIndexTaskIOConfig backfillIoConfig = new KafkaIndexTaskIOConfig( - 0, - baseSequenceName, - null, - null, - new SeekableStreamStartSequenceNumbers<>(kafkaIoConfig.getStream(), startOffsets, Collections.emptySet()), - new SeekableStreamEndSequenceNumbers<>(kafkaIoConfig.getStream(), endOffsets), - kafkaIoConfig.getConsumerProperties(), - kafkaIoConfig.getPollTimeout(), - false, // useTransaction = false for backfill (no supervisor coordination) - null, // minimumMessageTime - no time filtering for backfill - null, // maximumMessageTime - no time filtering for backfill - kafkaIoConfig.getInputFormat(), - kafkaIoConfig.getConfigOverrides(), - kafkaIoConfig.isMultiTopic(), - null, // refreshRejectionPeriodsInMinutes - don't refresh rejection periods for backfill - false // supervised = false - ); + // Get the backfillTaskCount from config + int backfillTaskCount = spec.getIoConfig().getBackfillTaskCount(); + List partitions = new ArrayList<>(startOffsets.keySet()); - // Create backfill task with different supervisorId - String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName); - Map context = createBaseTaskContexts(); - // Use APPEND locks to allow writing to intervals that may overlap with main supervisor - context.put("useConcurrentLocks", true); + // Determine actual number of tasks (can't have more tasks than partitions) + int numTasks = Math.min(backfillTaskCount, partitions.size()); - KafkaIndexTask backfillTask = new KafkaIndexTask( - taskId, - backfillSupervisorId, // Use backfill supervisorId instead of spec.getId() - new TaskResource(baseSequenceName, 1), - spec.getDataSchema(), - spec.getTuningConfig(), - backfillIoConfig, - context, - sortingMapper, - null // no server priority for backfill tasks + log.info( + "Submitting %d backfill task(s) with supervisorId[%s] for %d partition(s)", + numTasks, + backfillSupervisorId, + partitions.size() ); - Optional taskQueue = getTaskMaster().getTaskQueue(); - if (taskQueue.isPresent()) { - log.info( - "Submitting backfill task[%s] with supervisorId[%s] for offsets from [%s] to [%s]", + // Split partitions into groups for each task + int partitionsPerTask = partitions.size() / numTasks; + int remainder = partitions.size() % numTasks; + + int startIdx = 0; + for (int taskNum = 0; taskNum < numTasks; taskNum++) { + // Distribute remainder across first few tasks + int taskPartitionCount = partitionsPerTask + (taskNum < remainder ? 1 : 0); + int endIdx = startIdx + taskPartitionCount; + + List taskPartitions = partitions.subList(startIdx, endIdx); + + // Create offset maps for this task's partitions only + Map taskStartOffsets = new HashMap<>(); + Map taskEndOffsets = new HashMap<>(); + for (KafkaTopicPartition partition : taskPartitions) { + taskStartOffsets.put(partition, startOffsets.get(partition)); + taskEndOffsets.put(partition, endOffsets.get(partition)); + } + + String baseSequenceName = generateSequenceName( + taskStartOffsets, + null, // minimumMessageTime - process all data in range + null, // maximumMessageTime - process all data in range + spec.getDataSchema(), + spec.getTuningConfig() + ); + + KafkaSupervisorIOConfig kafkaIoConfig = spec.getIoConfig(); + KafkaIndexTaskIOConfig backfillIoConfig = new KafkaIndexTaskIOConfig( + taskNum, // taskGroupId + baseSequenceName, + null, + null, + new SeekableStreamStartSequenceNumbers<>(kafkaIoConfig.getStream(), taskStartOffsets, Collections.emptySet()), + new SeekableStreamEndSequenceNumbers<>(kafkaIoConfig.getStream(), taskEndOffsets), + kafkaIoConfig.getConsumerProperties(), + kafkaIoConfig.getPollTimeout(), + false, // useTransaction = false for backfill (no supervisor coordination) + null, // minimumMessageTime - no time filtering for backfill + null, // maximumMessageTime - no time filtering for backfill + kafkaIoConfig.getInputFormat(), + kafkaIoConfig.getConfigOverrides(), + kafkaIoConfig.isMultiTopic(), + null, // refreshRejectionPeriodsInMinutes - don't refresh rejection periods for backfill + false // supervised = false + ); + + // Create backfill task with different supervisorId + String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName); + Map context = createBaseTaskContexts(); + // Use APPEND locks to allow writing to intervals that may overlap with main supervisor + context.put("useConcurrentLocks", true); + + KafkaIndexTask backfillTask = new KafkaIndexTask( taskId, - backfillSupervisorId, - startOffsets, - endOffsets + backfillSupervisorId, // Use backfill supervisorId instead of spec.getId() + new TaskResource(baseSequenceName, 1), + spec.getDataSchema(), + spec.getTuningConfig(), + backfillIoConfig, + context, + sortingMapper, + null // no server priority for backfill tasks ); - taskQueue.get().add(backfillTask); - } else { - log.error("Failed to submit backfill task because I'm not the leader!"); + + Optional taskQueue = getTaskMaster().getTaskQueue(); + if (taskQueue.isPresent()) { + log.info( + "Submitting backfill task[%s] (task %d of %d) with supervisorId[%s] for partitions %s, offsets from [%s] to [%s]", + taskId, + taskNum + 1, + numTasks, + backfillSupervisorId, + taskPartitions, + taskStartOffsets, + taskEndOffsets + ); + taskQueue.get().add(backfillTask); + } else { + log.error("Failed to submit backfill task because I'm not the leader!"); + break; + } + + startIdx = endIdx; } } catch (Exception e) { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 992ff292694a..1f144e49638a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -78,7 +78,8 @@ public KafkaSupervisorIOConfig( @JsonProperty("idleConfig") IdleConfig idleConfig, @JsonProperty("stopTaskCount") Integer stopTaskCount, @Nullable @JsonProperty("emitTimeLagMetrics") Boolean emitTimeLagMetrics, - @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas + @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas, + @Nullable @JsonProperty("backfillTaskCount") Integer backfillTaskCount ) { super( @@ -98,7 +99,8 @@ public KafkaSupervisorIOConfig( lateMessageRejectionStartDateTime, idleConfig, stopTaskCount, - serverPriorityToReplicas + serverPriorityToReplicas, + backfillTaskCount ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java index 24c1656fc7e6..dd62c8385f45 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java @@ -93,6 +93,7 @@ public KafkaSupervisorIOConfig build() idleConfig, stopTaskCount, null, + null, null ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java index 6295d41937e8..049b9959088c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java @@ -341,6 +341,7 @@ public void testAutoScalingConfigSerde() throws JsonProcessingException null, null, false, + null, null ); String ioConfig = mapper.writeValueAsString(kafkaSupervisorIOConfig); @@ -377,6 +378,7 @@ public void testAutoScalingConfigSerde() throws JsonProcessingException null, null, false, + null, null ); Assert.assertEquals(5, kafkaSupervisorIOConfig.getTaskCount().intValue()); @@ -430,6 +432,7 @@ public void testIdleConfigSerde() throws JsonProcessingException mapper.convertValue(idleConfig, IdleConfig.class), null, false, + null, null ); String ioConfig = mapper.writeValueAsString(kafkaSupervisorIOConfig); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index b5e00bcaab4b..9e11053939d9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -5401,7 +5401,8 @@ public void testComputeUnassignedServerPriorities_whenMultipleReplicasPerPriorit Map.of( 10, 2, 20, 3 - ) + ), + null ); Assert.assertEquals(5, (int) kafkaSupervisorIOConfig.getReplicas()); @@ -5686,7 +5687,8 @@ private TestableKafkaSupervisor getTestableSupervisor( idleConfig, null, true, - serverPriorityToReplicas + serverPriorityToReplicas, + null ); KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( @@ -5781,6 +5783,7 @@ private TestableKafkaSupervisor getTestableSupervisorCustomIsTaskCurrent( null, null, false, + null, null ); @@ -5876,6 +5879,7 @@ private KafkaSupervisor createSupervisor( null, null, false, + null, null ); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index 6c325bd0744d..e1770f42b57d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -99,7 +99,8 @@ public KinesisSupervisorIOConfig( lateMessageRejectionStartDateTime, new IdleConfig(null, null), null, - serverPriorityToReplicas + serverPriorityToReplicas, + null ); this.endpoint = endpoint != null diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index 633bd9b70dc9..31f1781bb352 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -54,6 +54,7 @@ public abstract class SeekableStreamSupervisorIOConfig @Nullable private final IdleConfig idleConfig; @Nullable private final Integer stopTaskCount; @Nullable private final Map serverPriorityToReplicas; + @Nullable private final Integer backfillTaskCount; private final LagAggregator lagAggregator; private final boolean autoScalerEnabled; @@ -75,7 +76,8 @@ public SeekableStreamSupervisorIOConfig( DateTime lateMessageRejectionStartDateTime, @Nullable IdleConfig idleConfig, @Nullable Integer stopTaskCount, - @Nullable Map serverPriorityToReplicas + @Nullable Map serverPriorityToReplicas, + @Nullable Integer backfillTaskCount ) { this.stream = Preconditions.checkNotNull(stream, "stream cannot be null"); @@ -154,6 +156,9 @@ public SeekableStreamSupervisorIOConfig( } else { this.replicas = replicas != null ? replicas : 1; } + + // Default backfillTaskCount to taskCount / 2 + this.backfillTaskCount = backfillTaskCount != null ? backfillTaskCount : Math.max(1, this.taskCount / 2); } private static Duration defaultDuration(final Period period, final String theDefault) @@ -259,6 +264,12 @@ public Optional getLateMessageRejectionStartDateTime() return lateMessageRejectionStartDateTime; } + @JsonProperty + public Integer getBackfillTaskCount() + { + return backfillTaskCount; + } + @Nullable @JsonProperty public IdleConfig getIdleConfig() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java index 97456165f361..83771436822e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java @@ -346,6 +346,7 @@ private TestableSeekableStreamSupervisorIOConfig( lateMessageRejectionStartDateTime, idleConfig, null, + null, null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java index 3974b9bebca9..c0b7dc1d753a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java @@ -69,6 +69,7 @@ public void testAllDefaults() null, null, null, + null, null ) { @@ -121,6 +122,7 @@ public void testAutoScalerEnabledPreservesTaskCountWhenNonNull() null, null, null, + null, null ) { @@ -146,6 +148,7 @@ public void testAutoScalerEnabledPreservesTaskCountWhenNonNull() null, null, null, + null, null ) { @@ -178,6 +181,7 @@ public void testBothLateMessageRejectionPeriodAndStartDateTime() DateTimes.nowUtc(), null, null, + null, null ) { @@ -213,6 +217,7 @@ public void testNullAggregatorThrows() null, null, null, + null, null ) { @@ -246,6 +251,7 @@ public void testGetMaxAllowedStopsScalingDisabled() null, null, null, + null, null ) { @@ -270,6 +276,7 @@ public void testGetMaxAllowedStopsScalingDisabled() null, null, 3, + null, null ) { @@ -306,6 +313,7 @@ public void testGetMaxAllowedStopsScalingEnabled() null, null, 1, + null, null ) { @@ -339,6 +347,7 @@ public void testGetMaxAllowedStopsScalingEnabled() null, null, 1, + null, null ) { @@ -369,6 +378,7 @@ public void testGetMaxAllowedStopsScalingEnabled() null, null, null, + null, null ) { @@ -456,7 +466,8 @@ private SeekableStreamSupervisorIOConfig makeSeekableStreamSupervisorIOConfig(@N null, null, null, - serverPriorityToReplicas + serverPriorityToReplicas, + null ) { }; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java index baa5ca240500..27cc4d984e5e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java @@ -791,6 +791,7 @@ public void testSeekableStreamSupervisorSpecWithScaleDisable() throws Interrupte null, null, null, + null, null ) { @@ -849,6 +850,7 @@ public void testEnablingIdleBeviourPerSupervisorWithOverlordConfigEnabled() null, new IdleConfig(true, null), null, + null, null ) { @@ -1461,6 +1463,7 @@ private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scal null, null, null, + null, null ) { @@ -1483,6 +1486,7 @@ private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scal null, null, null, + null, null ) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 2fb102313bcf..780eadb10220 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -696,6 +696,7 @@ public void testIdleStateTransition() throws Exception null, new IdleConfig(true, 200L), null, + null, null ) { @@ -803,6 +804,7 @@ public void testIdleOnStartUpAndTurnsToRunningAfterLagUpdates() null, new IdleConfig(true, 200L), null, + null, null ) { @@ -1103,6 +1105,7 @@ public void testCheckpointForActiveTaskGroup() throws InterruptedException, Json null, new IdleConfig(true, 200L), null, + null, null ) {}; @@ -1322,6 +1325,7 @@ public void testEarlyStoppingOfTaskGroupBasedOnStopTaskCount() throws Interrupte null, new IdleConfig(true, 200L), stopTaskCount, + null, null ) { @@ -1558,6 +1562,7 @@ public void testSupervisorStopTaskGroupEarly() throws JsonProcessingException, I null, new IdleConfig(true, 200L), null, + null, null ) { @@ -2709,6 +2714,7 @@ private void expectEmitterSupervisor(boolean suspended) null, null, null, + null, null ) { @@ -2773,6 +2779,7 @@ public void testMaxAllowedStopsWithStopTaskCountRatio() null, null, 1, // ensure this is overridden + null, null ) { @@ -2928,7 +2935,8 @@ private static SeekableStreamSupervisorIOConfig createSupervisorIOConfig( null, null, null, - serverPriorityToReplicas + serverPriorityToReplicas, + null ) { }; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java index 1e3bb7bdc5b9..75b8409c472b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java @@ -538,6 +538,7 @@ protected SeekableStreamSupervisorIOConfig createIOConfig(int taskCount, CostBas null, null, null, + null, null ) { From ca99b9422959175794fa591434ce6013ac0f2879 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 20 Mar 2026 12:46:38 -0700 Subject: [PATCH 10/27] Rename skippedOffsetRange to backfillRange --- .../overlord/supervisor/SupervisorManager.java | 12 ++++++------ .../overlord/supervisor/SupervisorResource.java | 10 +++++----- .../supervisor/SeekableStreamSupervisor.java | 4 ++-- .../overlord/supervisor/SupervisorResourceTest.java | 6 +++--- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 19a3d9c231d4..47446069e014 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -369,17 +369,17 @@ public boolean resetSupervisor(String id, @Nullable DataSourceMetadata resetData } /** - * Resets a supervisor to latest offsets and returns the skipped offset ranges. + * Resets a supervisor to latest offsets and backfills the skipped offset ranges. * Requirements: * - Supervisor must be RUNNING - needs active stream connection - * - Supervisor must be a SeekableStreamSupervisor (Kafka, Kinesis) + * - Supervisor must be a SeekableStreamSupervisor * - useEarliestOffset must be false (otherwise supervisor always starts from earliest) * @param id supervisor ID * @return Map containing supervisorId and skipped offset ranges * @throws IllegalArgumentException if supervisor doesn't exist or if useEarliestOffset is true * @throws IllegalStateException if supervisor is not running */ - public Map resetSupervisorAndReturnSkippedOffsets(String id) + public Map resetSupervisorAndBackfill(String id) { Preconditions.checkState(started, "SupervisorManager not started"); Preconditions.checkNotNull(id, "id"); @@ -424,15 +424,15 @@ public Map resetSupervisorAndReturnSkippedOffsets(String id) autoscaler.reset(); } - Map skippedRanges = streamSupervisor.calculateSkippedOffsetRanges(startOffsets, latestOffsets); + Map backfillRange = streamSupervisor.calculateBackfillRange(startOffsets, latestOffsets); streamSupervisor.submitBackfillTask(startOffsets, latestOffsets); - log.info("Successfully reset supervisor[%s] to latest. Skipped ranges: %s", id, skippedRanges); + log.info("Successfully reset supervisor[%s] to latest. Skipped ranges: %s", id, backfillRange); return ImmutableMap.of( "id", id, - "skippedOffsets", skippedRanges + "backfillRange", backfillRange ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index b47faa886e18..219c09b1aba9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -594,11 +594,11 @@ public Response specGetHistory( @ResourceFilters(SupervisorResourceFilter.class) public Response reset( @PathParam("id") final String id, - @QueryParam("returnSkippedOffsets") Boolean returnSkippedOffsets + @QueryParam("backfill") Boolean backfill ) { - if (Boolean.TRUE.equals(returnSkippedOffsets)) { - return handleResetAndReturnSkippedOffsets(id); + if (Boolean.TRUE.equals(backfill)) { + return handleResetAndBackfill(id); } else { return handleResetRequest(id, null); } @@ -635,12 +635,12 @@ private Response handleResetRequest( ); } - private Response handleResetAndReturnSkippedOffsets(final String id) + private Response handleResetAndBackfill(final String id) { return asLeaderWithSupervisorManager( manager -> { try { - Map result = manager.resetSupervisorAndReturnSkippedOffsets(id); + Map result = manager.resetSupervisorAndBackfill(id); return Response.ok(result).build(); } catch (IllegalArgumentException e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 32256b16eef2..5b68ec0be4b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1917,14 +1917,14 @@ public void gracefulShutdownInternal() throws ExecutionException, InterruptedExc } /** - * Calculates the skipped offset ranges between start and end offsets. + * Calculates the backfill range between start and end offsets. * Returns a map with partition ID as key and [startOffset, endOffset] array as value. * * @param startOffsets Starting offsets (last checkpointed) * @param endOffsets Ending offsets (latest from stream) * @return Map of partition ID to offset range [start, end] */ - public Map calculateSkippedOffsetRanges( + public Map calculateBackfillRange( Map startOffsets, Map endOffsets ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 584d52749fcf..a3d13f718671 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -1328,12 +1328,12 @@ public void testResetOffsets() } @Test - public void testResetOffsetsAndReturnSkippedOffsets() + public void testResetOffsetsAndBackfill() { Capture id1 = Capture.newInstance(); Map expectedResult = ImmutableMap.of( "id", "my-id", - "skippedOffsets", ImmutableMap.of( + "backfillRange", ImmutableMap.of( "0", ImmutableMap.of("start", 50L, "end", 100L), "1", ImmutableMap.of("start", 150L, "end", 200L), "2", ImmutableMap.of("start", 250L, "end", 300L) @@ -1341,7 +1341,7 @@ public void testResetOffsetsAndReturnSkippedOffsets() ); EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); - EasyMock.expect(supervisorManager.resetSupervisorAndReturnSkippedOffsets( + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill( EasyMock.capture(id1) )).andReturn(expectedResult); replayAll(); From 1f88656382b2f286171627edb6f06e8de66d1097 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 20 Mar 2026 13:02:27 -0700 Subject: [PATCH 11/27] Null check for startOffset --- .../druid/indexing/kafka/supervisor/KafkaSupervisor.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 61d63b683977..55d51ce23c10 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -279,7 +279,7 @@ public void submitBackfillTask( // Get the backfillTaskCount from config int backfillTaskCount = spec.getIoConfig().getBackfillTaskCount(); - List partitions = new ArrayList<>(startOffsets.keySet()); + List partitions = new ArrayList<>(endOffsets.keySet()); // Determine actual number of tasks (can't have more tasks than partitions) int numTasks = Math.min(backfillTaskCount, partitions.size()); @@ -307,7 +307,12 @@ public void submitBackfillTask( Map taskStartOffsets = new HashMap<>(); Map taskEndOffsets = new HashMap<>(); for (KafkaTopicPartition partition : taskPartitions) { - taskStartOffsets.put(partition, startOffsets.get(partition)); + Long startOffset = startOffsets.get(partition); + if (startOffset == null) { + log.info("No checkpoint has occurred before for partition [%s], setting startOffset equal to endOffset to skip data consumption", partition); + startOffset = endOffsets.get(partition); + } + taskStartOffsets.put(partition, startOffset); taskEndOffsets.put(partition, endOffsets.get(partition)); } From 20e3cae8c27a7f0a9121418ef9bb29703864ef64 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 20 Mar 2026 14:19:14 -0700 Subject: [PATCH 12/27] Validate the main supervisor has useConcurrentLocks set to true --- .../indexing/overlord/supervisor/SupervisorManager.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 47446069e014..aea78ddd6a61 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -392,12 +392,20 @@ public Map resetSupervisorAndBackfill(String id) throw new IAE("Supervisor[%s] is not a SeekableStreamSupervisor", id); } SeekableStreamSupervisor streamSupervisor = (SeekableStreamSupervisor) supervisorPair.lhs; + SeekableStreamSupervisorSpec streamSpec = (SeekableStreamSupervisorSpec) supervisorPair.rhs; // Verify useEarliestOffset is false if (streamSupervisor.getIoConfig().isUseEarliestSequenceNumber()) { throw new IAE("Reset with skipped offsets is not supported when useEarliestOffset is true."); } + // Verify useConcurrentLocks is enabled + if (streamSpec.getContext() == null || !Boolean.TRUE.equals(streamSpec.getContext().get("useConcurrentLocks"))) { + throw new IAE( + "Backfill tasks require 'useConcurrentLocks' to be set to true in the supervisor context to allow concurrent writes with the main supervisor tasks" + ); + } + // We need an active recordSupplier to query the latest offsets from the stream if (supervisorPair.lhs.getState() != SupervisorStateManager.BasicState.RUNNING) { throw new ISE("A running supervisor is required to query the latest offsets from the stream"); From ef17672541349418eb757de346219bb395dc73b4 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 20 Mar 2026 14:55:57 -0700 Subject: [PATCH 13/27] javadoc update --- .../druid/indexing/overlord/supervisor/SupervisorManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index aea78ddd6a61..f7bbe1a6e24e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -374,6 +374,7 @@ public boolean resetSupervisor(String id, @Nullable DataSourceMetadata resetData * - Supervisor must be RUNNING - needs active stream connection * - Supervisor must be a SeekableStreamSupervisor * - useEarliestOffset must be false (otherwise supervisor always starts from earliest) + * - useConcurrentLocks must be true * @param id supervisor ID * @return Map containing supervisorId and skipped offset ranges * @throws IllegalArgumentException if supervisor doesn't exist or if useEarliestOffset is true From e3375d8d6e69ce0af9a3944d5d121c6a572811c5 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 25 Mar 2026 15:49:06 -0700 Subject: [PATCH 14/27] Tweak logging --- .../druid/indexing/overlord/supervisor/SupervisorManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 338a62a37f53..4986f338488b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -467,7 +467,7 @@ public Map resetSupervisorAndBackfill(String id) streamSupervisor.submitBackfillTask(startOffsets, latestOffsets); - log.info("Successfully reset supervisor[%s] to latest. Skipped ranges: %s", id, backfillRange); + log.info("Successfully reset supervisor[%s] to latest. Backfill range: %s", id, backfillRange); return ImmutableMap.of( "id", id, From 87d75d3a1a1cf28b010734a5e87b12ba5e9387fe Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 25 Mar 2026 22:13:31 -0700 Subject: [PATCH 15/27] Address deprecation notices --- .../indexing/kafka/supervisor/KafkaSupervisor.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 6510c23c6a65..1aad48682563 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -280,10 +280,10 @@ public void submitBackfillTask( } try { - String backfillSupervisorId = spec.getDataSchema().getDataSource() + "_backfill"; + String backfillSupervisorId = spec.getSpec().getDataSchema().getDataSource() + "_backfill"; // Get the backfillTaskCount from config - int backfillTaskCount = spec.getIoConfig().getBackfillTaskCount(); + int backfillTaskCount = spec.getSpec().getIOConfig().getBackfillTaskCount(); List partitions = new ArrayList<>(endOffsets.keySet()); // Determine actual number of tasks (can't have more tasks than partitions) @@ -325,11 +325,11 @@ public void submitBackfillTask( taskStartOffsets, null, // minimumMessageTime - process all data in range null, // maximumMessageTime - process all data in range - spec.getDataSchema(), - spec.getTuningConfig() + spec.getSpec().getDataSchema(), + spec.getSpec().getTuningConfig() ); - KafkaSupervisorIOConfig kafkaIoConfig = spec.getIoConfig(); + KafkaSupervisorIOConfig kafkaIoConfig = spec.getSpec().getIOConfig(); KafkaIndexTaskIOConfig backfillIoConfig = new KafkaIndexTaskIOConfig( taskNum, // taskGroupId baseSequenceName, @@ -359,8 +359,8 @@ public void submitBackfillTask( taskId, backfillSupervisorId, // Use backfill supervisorId instead of spec.getId() new TaskResource(baseSequenceName, 1), - spec.getDataSchema(), - spec.getTuningConfig(), + spec.getSpec().getDataSchema(), + spec.getSpec().getTuningConfig(), backfillIoConfig, context, sortingMapper, From fb0b041aa28f977d03917b925c175f8d875f17de Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Thu, 26 Mar 2026 13:54:39 -0700 Subject: [PATCH 16/27] Use existing useTransaction to disable checkpointing instead of new supervised field --- .../rabbitstream/RabbitStreamIndexTaskIOConfig.java | 6 ++---- .../supervisor/RabbitStreamSupervisor.java | 3 +-- .../druid/indexing/kafka/KafkaIndexTaskIOConfig.java | 9 +++------ .../indexing/kafka/supervisor/KafkaSupervisor.java | 6 ++---- .../indexing/kinesis/KinesisIndexTaskIOConfig.java | 9 +++------ .../SeekableStreamIndexTaskIOConfig.java | 12 +----------- .../SeekableStreamIndexTaskRunner.java | 6 +++--- .../supervisor/SeekableStreamSupervisor.java | 2 +- .../TestSeekableStreamIndexTaskIOConfig.java | 3 +-- .../SeekableStreamSupervisorStateTest.java | 6 ++---- .../supervisor/SeekableStreamSupervisorTestBase.java | 3 +-- .../test/java/org/apache/druid/cli/CliPeonTest.java | 3 +-- 12 files changed, 21 insertions(+), 47 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java index 30780155ce67..721e66f6f3af 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java @@ -54,8 +54,7 @@ public RabbitStreamIndexTaskIOConfig( @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("uri") String uri, - @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes, - @JsonProperty("supervised") @Nullable Boolean supervised + @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes ) { super( @@ -67,8 +66,7 @@ public RabbitStreamIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes, - supervised + refreshRejectionPeriodsInMinutes ); this.pollTimeout = pollTimeout != null ? pollTimeout : RabbitStreamSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS; diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java index f5241c1300c0..32ac2b9c8651 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java @@ -203,8 +203,7 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( maximumMessageTime, ioConfig.getInputFormat(), rabbitConfig.getUri(), - ioConfig.getTaskDuration().getStandardMinutes(), - true + ioConfig.getTaskDuration().getStandardMinutes() ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java index d70864d1f22b..07c0f80fbe83 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java @@ -64,8 +64,7 @@ public KafkaIndexTaskIOConfig( @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("configOverrides") @Nullable KafkaConfigOverrides configOverrides, @JsonProperty("multiTopic") @Nullable Boolean multiTopic, - @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes, - @JsonProperty("supervised") @Nullable Boolean supervised + @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes ) { super( @@ -79,8 +78,7 @@ public KafkaIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes, - supervised + refreshRejectionPeriodsInMinutes ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); @@ -130,8 +128,7 @@ public KafkaIndexTaskIOConfig( inputFormat, configOverrides, KafkaSupervisorIOConfig.DEFAULT_IS_MULTI_TOPIC, - refreshRejectionPeriodsInMinutes, - true + refreshRejectionPeriodsInMinutes ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 1aad48682563..bc79a404e9b2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -229,8 +229,7 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ioConfig.getInputFormat(), kafkaIoConfig.getConfigOverrides(), kafkaIoConfig.isMultiTopic(), - ioConfig.getTaskDuration().getStandardMinutes(), - true + ioConfig.getTaskDuration().getStandardMinutes() ); } @@ -345,8 +344,7 @@ public void submitBackfillTask( kafkaIoConfig.getInputFormat(), kafkaIoConfig.getConfigOverrides(), kafkaIoConfig.isMultiTopic(), - null, // refreshRejectionPeriodsInMinutes - don't refresh rejection periods for backfill - false // supervised = false + null // refreshRejectionPeriodsInMinutes - don't refresh rejection periods for backfill ); // Create backfill task with different supervisorId diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java index 0044e7cfb4a0..2df59bbca35f 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java @@ -79,8 +79,7 @@ public KinesisIndexTaskIOConfig( @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, - @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes, - @JsonProperty("supervised") @Nullable Boolean supervised + @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes ) { super( @@ -92,8 +91,7 @@ public KinesisIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes, - supervised + refreshRejectionPeriodsInMinutes ); Preconditions.checkArgument( getEndSequenceNumbers().getPartitionSequenceNumberMap() @@ -141,8 +139,7 @@ public KinesisIndexTaskIOConfig( fetchDelayMillis, awsAssumedRoleArn, awsExternalId, - refreshRejectionPeriodsInMinutes, - true + refreshRejectionPeriodsInMinutes ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java index 7681bb78fc2b..8f92f76e8360 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java @@ -32,7 +32,6 @@ public abstract class SeekableStreamIndexTaskIOConfig implements IOConfig { private static final boolean DEFAULT_USE_TRANSACTION = true; - private static final boolean DEFAULT_SUPERVISED = true; private final Integer taskGroupId; private final String baseSequenceName; @@ -43,7 +42,6 @@ public abstract class SeekableStreamIndexTaskIOConfig Date: Thu, 26 Mar 2026 15:29:57 -0700 Subject: [PATCH 17/27] Test coverage for calculateBackfillRange --- .../supervisor/SupervisorManager.java | 2 +- .../SeekableStreamSupervisorTest.java | 210 ++++++++++++++++++ 2 files changed, 211 insertions(+), 1 deletion(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 4986f338488b..d374f9797875 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -401,10 +401,10 @@ public boolean resetSupervisor(String id, @Nullable DataSourceMetadata resetData /** * Resets a supervisor to latest offsets and backfills the skipped offset ranges. * Requirements: - * - Supervisor must be RUNNING - needs active stream connection * - Supervisor must be a SeekableStreamSupervisor * - useEarliestOffset must be false (otherwise supervisor always starts from earliest) * - useConcurrentLocks must be true + * - Supervisor must be RUNNING - needs active stream connection * @param id supervisor ID * @return Map containing supervisorId and skipped offset ranges * @throws IllegalArgumentException if supervisor doesn't exist or if useEarliestOffset is true diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java new file mode 100644 index 000000000000..b9e830c26dd6 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.seekablestream.supervisor; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class SeekableStreamSupervisorTest +{ + @Test + public void testCalculateBackfillRange_withAllOffsetsPresent() + { + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + Map startOffsets = ImmutableMap.of( + "partition0", 100L, + "partition1", 200L, + "partition2", 300L + ); + + Map endOffsets = ImmutableMap.of( + "partition0", 150L, + "partition1", 250L, + "partition2", 350L + ); + + Map result = supervisor.calculateBackfillRange(startOffsets, endOffsets); + + Assert.assertEquals(3, result.size()); + Assert.assertEquals( + ImmutableMap.of("start", 100L, "end", 150L), + result.get("partition0") + ); + Assert.assertEquals( + ImmutableMap.of("start", 200L, "end", 250L), + result.get("partition1") + ); + Assert.assertEquals( + ImmutableMap.of("start", 300L, "end", 350L), + result.get("partition2") + ); + } + + @Test + public void testCalculateBackfillRange_withMissingStartOffsets() + { + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + // Only partition0 has a checkpoint + Map startOffsets = ImmutableMap.of( + "partition0", 100L + ); + + Map endOffsets = ImmutableMap.of( + "partition0", 150L, + "partition1", 250L, + "partition2", 350L + ); + + Map result = supervisor.calculateBackfillRange(startOffsets, endOffsets); + + Assert.assertEquals(3, result.size()); + + // Partition 0 has both offsets + Assert.assertEquals( + ImmutableMap.of("start", 100L, "end", 150L), + result.get("partition0") + ); + + // Partition 1 and 2 have no checkpoint + Map partition1Result = (Map) result.get("partition1"); + Assert.assertEquals("none", partition1Result.get("start")); + Assert.assertEquals(250L, partition1Result.get("end")); + Assert.assertEquals("No committed offset found for this partition", partition1Result.get("note")); + + Map partition2Result = (Map) result.get("partition2"); + Assert.assertEquals("none", partition2Result.get("start")); + Assert.assertEquals(350L, partition2Result.get("end")); + Assert.assertEquals("No committed offset found for this partition", partition2Result.get("note")); + } + + @Test + public void testCalculateBackfillRange_withNullStartOffsets() + { + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + Map endOffsets = ImmutableMap.of( + "partition0", 150L, + "partition1", 250L, + "partition2", 350L + ); + + Map result = supervisor.calculateBackfillRange(null, endOffsets); + + Assert.assertEquals(3, result.size()); + for (Map.Entry entry : endOffsets.entrySet()) { + Map partitionResult = (Map) result.get(entry.getKey()); + Assert.assertEquals("none", partitionResult.get("start")); + Assert.assertEquals(entry.getValue(), partitionResult.get("end")); + Assert.assertEquals("No committed offset found for this partition", partitionResult.get("note")); + } + } + + /** + * Minimal test implementation of SeekableStreamSupervisor to test calculateBackfillRange + */ + private static class TestSeekableStreamSupervisor extends SeekableStreamSupervisor + { + public TestSeekableStreamSupervisor() + { + super( + null, + null, + null, + null, + null, + null, + null, + false + ); + } + + @Override + protected int getTaskGroupIdForPartition(String partition) + { + return 0; + } + + @Override + protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) + { + return false; + } + + @Override + protected boolean doesTaskTypeMatchSupervisor(Task task) + { + return false; + } + + @Override + protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( + String stream, + Map map + ) + { + return null; + } + + @Override + protected OrderedSequenceNumber makeSequenceNumber(Long seq, boolean isExclusive) + { + return null; + } + + @Override + protected Long getNotSetMarker() + { + return null; + } + + @Override + protected Long getEndOfPartitionMarker() + { + return null; + } + + @Override + protected boolean isEndOfShard(Long seqNum) + { + return false; + } + + @Override + protected boolean isShardExpirationMarker(Long seqNum) + { + return false; + } + + @Override + protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() + { + return false; + } + } +} From ee2e619b297e9fa22cd6980e4cfb363a91890a3f Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Thu, 26 Mar 2026 16:25:31 -0700 Subject: [PATCH 18/27] Unit test fixes --- .../SeekableStreamSupervisorTest.java | 134 +++--------------- 1 file changed, 23 insertions(+), 111 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java index b9e830c26dd6..c82461031b8d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java @@ -20,32 +20,28 @@ package org.apache.druid.indexing.seekablestream.supervisor; import com.google.common.collect.ImmutableMap; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; -import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.junit.Assert; import org.junit.Test; import java.util.Map; -public class SeekableStreamSupervisorTest +public class SeekableStreamSupervisorTest extends SeekableStreamSupervisorTestBase { @Test public void testCalculateBackfillRange_withAllOffsetsPresent() { - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); - Map startOffsets = ImmutableMap.of( - "partition0", 100L, - "partition1", 200L, - "partition2", 300L + Map startOffsets = ImmutableMap.of( + "0", "100", + "1", "200", + "2", "300" ); - Map endOffsets = ImmutableMap.of( - "partition0", 150L, - "partition1", 250L, - "partition2", 350L + Map endOffsets = ImmutableMap.of( + "0", "150", + "1", "250", + "2", "350" ); Map result = supervisor.calculateBackfillRange(startOffsets, endOffsets); @@ -68,17 +64,16 @@ public void testCalculateBackfillRange_withAllOffsetsPresent() @Test public void testCalculateBackfillRange_withMissingStartOffsets() { - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); - // Only partition0 has a checkpoint - Map startOffsets = ImmutableMap.of( - "partition0", 100L + Map startOffsets = ImmutableMap.of( + "0", "100" ); - Map endOffsets = ImmutableMap.of( - "partition0", 150L, - "partition1", 250L, - "partition2", 350L + Map endOffsets = ImmutableMap.of( + "0", "150", + "1", "250", + "2", "350" ); Map result = supervisor.calculateBackfillRange(startOffsets, endOffsets); @@ -106,105 +101,22 @@ public void testCalculateBackfillRange_withMissingStartOffsets() @Test public void testCalculateBackfillRange_withNullStartOffsets() { - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); - Map endOffsets = ImmutableMap.of( - "partition0", 150L, - "partition1", 250L, - "partition2", 350L + Map endOffsets = ImmutableMap.of( + "0", "150", + "1", "250", + "2", "350" ); Map result = supervisor.calculateBackfillRange(null, endOffsets); Assert.assertEquals(3, result.size()); - for (Map.Entry entry : endOffsets.entrySet()) { + for (Map.Entry entry : endOffsets.entrySet()) { Map partitionResult = (Map) result.get(entry.getKey()); Assert.assertEquals("none", partitionResult.get("start")); Assert.assertEquals(entry.getValue(), partitionResult.get("end")); Assert.assertEquals("No committed offset found for this partition", partitionResult.get("note")); } } - - /** - * Minimal test implementation of SeekableStreamSupervisor to test calculateBackfillRange - */ - private static class TestSeekableStreamSupervisor extends SeekableStreamSupervisor - { - public TestSeekableStreamSupervisor() - { - super( - null, - null, - null, - null, - null, - null, - null, - false - ); - } - - @Override - protected int getTaskGroupIdForPartition(String partition) - { - return 0; - } - - @Override - protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) - { - return false; - } - - @Override - protected boolean doesTaskTypeMatchSupervisor(Task task) - { - return false; - } - - @Override - protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( - String stream, - Map map - ) - { - return null; - } - - @Override - protected OrderedSequenceNumber makeSequenceNumber(Long seq, boolean isExclusive) - { - return null; - } - - @Override - protected Long getNotSetMarker() - { - return null; - } - - @Override - protected Long getEndOfPartitionMarker() - { - return null; - } - - @Override - protected boolean isEndOfShard(Long seqNum) - { - return false; - } - - @Override - protected boolean isShardExpirationMarker(Long seqNum) - { - return false; - } - - @Override - protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() - { - return false; - } - } } From 8a5a61eca5901211bf6acf7ef04383ebfa383479 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 27 Mar 2026 09:10:38 -0700 Subject: [PATCH 19/27] Refactoring and Unit tests for calculateBackfillRange --- .../supervisor/SupervisorManager.java | 50 ++++++- .../supervisor/SeekableStreamSupervisor.java | 45 ------- .../supervisor/SupervisorManagerTest.java | 109 ++++++++++++++++ .../SeekableStreamSupervisorTest.java | 122 ------------------ 4 files changed, 158 insertions(+), 168 deletions(-) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index d374f9797875..1b7aecea27ee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -55,6 +56,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -463,7 +465,7 @@ public Map resetSupervisorAndBackfill(String id) autoscaler.reset(); } - Map backfillRange = streamSupervisor.calculateBackfillRange(startOffsets, latestOffsets); + Map backfillRange = calculateBackfillRange(startOffsets, latestOffsets); streamSupervisor.submitBackfillTask(startOffsets, latestOffsets); @@ -475,6 +477,52 @@ public Map resetSupervisorAndBackfill(String id) ); } + /** + * Calculates the backfill range between start and end offsets for display purposes. + * Returns a map with partition ID as key and offset range details as value. + * + * @param startOffsets Starting offsets (last checkpointed) + * @param endOffsets Ending offsets (latest from stream) + * @return Map of partition ID to offset range [start, end] + */ + @VisibleForTesting + public Map calculateBackfillRange( + Map startOffsets, + Map endOffsets + ) + { + Map backfillRange = new HashMap<>(); + + for (Map.Entry entry : endOffsets.entrySet()) { + Object partition = entry.getKey(); + Object endOffset = entry.getValue(); + Object startOffset = (startOffsets != null) ? startOffsets.get(partition) : null; + + if (startOffset != null) { + // Both start and end exist - calculate range + backfillRange.put( + partition, + ImmutableMap.of( + "start", startOffset, + "end", endOffset + ) + ); + } else { + // No checkpoint exists for this partition + backfillRange.put( + partition, + ImmutableMap.of( + "start", "none", + "end", endOffset, + "note", "No committed offset found for this partition" + ) + ); + } + } + + return backfillRange; + } + public boolean checkPointDataSourceMetadata( String supervisorId, int taskGroupId, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 6ef185eddcea..bf7f1dc6974a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -1926,51 +1926,6 @@ public void gracefulShutdownInternal() throws ExecutionException, InterruptedExc checkTaskDuration(); } - /** - * Calculates the backfill range between start and end offsets. - * Returns a map with partition ID as key and [startOffset, endOffset] array as value. - * - * @param startOffsets Starting offsets (last checkpointed) - * @param endOffsets Ending offsets (latest from stream) - * @return Map of partition ID to offset range [start, end] - */ - public Map calculateBackfillRange( - Map startOffsets, - Map endOffsets - ) - { - Map skippedRanges = new HashMap<>(); - - for (Entry entry : endOffsets.entrySet()) { - PartitionIdType partition = entry.getKey(); - SequenceOffsetType endOffset = entry.getValue(); - SequenceOffsetType startOffset = (startOffsets != null) ? startOffsets.get(partition) : null; - - if (startOffset != null) { - // Both start and end exist - calculate range - skippedRanges.put( - partition, - ImmutableMap.of( - "start", startOffset, - "end", endOffset - ) - ); - } else { - // No checkpoint exists for this partition - skippedRanges.put( - partition, - ImmutableMap.of( - "start", "none", - "end", endOffset, - "note", "No committed offset found for this partition" - ) - ); - } - } - - return skippedRanges; - } - @VisibleForTesting public void resetInternal(DataSourceMetadata dataSourceMetadata) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index 525444e23dea..60bc69159588 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -1068,6 +1068,115 @@ public void test_isAnotherTaskGroupPublishingToPartitions() ); } + @Test + public void testCalculateBackfillRangeWithBothStartAndEndOffsets() throws Exception + { + Map startOffsets = ImmutableMap.of( + "0", 100L, + "1", 200L, + "2", 300L + ); + Map endOffsets = ImmutableMap.of( + "0", 150L, + "1", 250L, + "2", 350L + ); + + Map result = manager.calculateBackfillRange(startOffsets, endOffsets); + + Assert.assertEquals(3, result.size()); + Assert.assertEquals(ImmutableMap.of("start", 100L, "end", 150L), result.get("0")); + Assert.assertEquals(ImmutableMap.of("start", 200L, "end", 250L), result.get("1")); + Assert.assertEquals(ImmutableMap.of("start", 300L, "end", 350L), result.get("2")); + } + + @Test + public void testCalculateBackfillRangeWithNullStartOffsets() throws Exception + { + Map endOffsets = ImmutableMap.of( + "0", 150L, + "1", 250L + ); + + Map result = manager.calculateBackfillRange(null, endOffsets); + + Assert.assertEquals(2, result.size()); + Map partition0 = (Map) result.get("0"); + Assert.assertEquals("none", partition0.get("start")); + Assert.assertEquals(150L, partition0.get("end")); + Assert.assertEquals("No committed offset found for this partition", partition0.get("note")); + + Map partition1 = (Map) result.get("1"); + Assert.assertEquals("none", partition1.get("start")); + Assert.assertEquals(250L, partition1.get("end")); + Assert.assertEquals("No committed offset found for this partition", partition1.get("note")); + } + + @Test + public void testCalculateBackfillRangeWithMissingStartOffsetsForSomePartitions() throws Exception + { + Map startOffsets = ImmutableMap.of( + "0", 100L, + "2", 300L + ); + Map endOffsets = ImmutableMap.of( + "0", 150L, + "1", 250L, + "2", 350L + ); + + Map result = manager.calculateBackfillRange(startOffsets, endOffsets); + + Assert.assertEquals(3, result.size()); + + // Partition 0 has start offset + Assert.assertEquals(ImmutableMap.of("start", 100L, "end", 150L), result.get("0")); + + // Partition 1 has no start offset + Map partition1 = (Map) result.get("1"); + Assert.assertEquals("none", partition1.get("start")); + Assert.assertEquals(250L, partition1.get("end")); + Assert.assertEquals("No committed offset found for this partition", partition1.get("note")); + + // Partition 2 has start offset + Assert.assertEquals(ImmutableMap.of("start", 300L, "end", 350L), result.get("2")); + } + + @Test + public void testCalculateBackfillRangeWithEmptyOffsets() throws Exception + { + Map startOffsets = ImmutableMap.of(); + Map endOffsets = ImmutableMap.of(); + + Map result = manager.calculateBackfillRange(startOffsets, endOffsets); + + Assert.assertEquals(0, result.size()); + } + + @Test + public void testCalculateBackfillRangeWithEmptyStartOffsets() throws Exception + { + Map startOffsets = ImmutableMap.of(); + Map endOffsets = ImmutableMap.of( + "0", 150L, + "1", 250L + ); + + Map result = manager.calculateBackfillRange(startOffsets, endOffsets); + + Assert.assertEquals(2, result.size()); + + Map partition0 = (Map) result.get("0"); + Assert.assertEquals("none", partition0.get("start")); + Assert.assertEquals(150L, partition0.get("end")); + Assert.assertEquals("No committed offset found for this partition", partition0.get("note")); + + Map partition1 = (Map) result.get("1"); + Assert.assertEquals("none", partition1.get("start")); + Assert.assertEquals(250L, partition1.get("end")); + Assert.assertEquals("No committed offset found for this partition", partition1.get("note")); + } + private static class TestSupervisorSpec implements SupervisorSpec { private final String id; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java deleted file mode 100644 index c82461031b8d..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTest.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.seekablestream.supervisor; - -import com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Map; - -public class SeekableStreamSupervisorTest extends SeekableStreamSupervisorTestBase -{ - @Test - public void testCalculateBackfillRange_withAllOffsetsPresent() - { - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); - - Map startOffsets = ImmutableMap.of( - "0", "100", - "1", "200", - "2", "300" - ); - - Map endOffsets = ImmutableMap.of( - "0", "150", - "1", "250", - "2", "350" - ); - - Map result = supervisor.calculateBackfillRange(startOffsets, endOffsets); - - Assert.assertEquals(3, result.size()); - Assert.assertEquals( - ImmutableMap.of("start", 100L, "end", 150L), - result.get("partition0") - ); - Assert.assertEquals( - ImmutableMap.of("start", 200L, "end", 250L), - result.get("partition1") - ); - Assert.assertEquals( - ImmutableMap.of("start", 300L, "end", 350L), - result.get("partition2") - ); - } - - @Test - public void testCalculateBackfillRange_withMissingStartOffsets() - { - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); - - Map startOffsets = ImmutableMap.of( - "0", "100" - ); - - Map endOffsets = ImmutableMap.of( - "0", "150", - "1", "250", - "2", "350" - ); - - Map result = supervisor.calculateBackfillRange(startOffsets, endOffsets); - - Assert.assertEquals(3, result.size()); - - // Partition 0 has both offsets - Assert.assertEquals( - ImmutableMap.of("start", 100L, "end", 150L), - result.get("partition0") - ); - - // Partition 1 and 2 have no checkpoint - Map partition1Result = (Map) result.get("partition1"); - Assert.assertEquals("none", partition1Result.get("start")); - Assert.assertEquals(250L, partition1Result.get("end")); - Assert.assertEquals("No committed offset found for this partition", partition1Result.get("note")); - - Map partition2Result = (Map) result.get("partition2"); - Assert.assertEquals("none", partition2Result.get("start")); - Assert.assertEquals(350L, partition2Result.get("end")); - Assert.assertEquals("No committed offset found for this partition", partition2Result.get("note")); - } - - @Test - public void testCalculateBackfillRange_withNullStartOffsets() - { - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); - - Map endOffsets = ImmutableMap.of( - "0", "150", - "1", "250", - "2", "350" - ); - - Map result = supervisor.calculateBackfillRange(null, endOffsets); - - Assert.assertEquals(3, result.size()); - for (Map.Entry entry : endOffsets.entrySet()) { - Map partitionResult = (Map) result.get(entry.getKey()); - Assert.assertEquals("none", partitionResult.get("start")); - Assert.assertEquals(entry.getValue(), partitionResult.get("end")); - Assert.assertEquals("No committed offset found for this partition", partitionResult.get("note")); - } - } -} From d46b5c8f087226be5875af6f11caaae8f6a39e6c Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 27 Mar 2026 11:17:35 -0700 Subject: [PATCH 20/27] Add test cases for resetSupervisorAndBackfill --- .../supervisor/SupervisorManagerTest.java | 223 ++++++++++++++++++ 1 file changed, 223 insertions(+) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index 60bc69159588..a8f8023906f6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.TestSeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -684,6 +685,228 @@ public void testResetSupervisorWithSpecificOffsets() verifyAll(); } + @Test + public void testResetSupervisorAndBackfill() + { + // Create mock SeekableStreamSupervisor and spec + SeekableStreamSupervisor streamSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); + SeekableStreamSupervisorSpec streamSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); + SeekableStreamSupervisorIOConfig ioConfig = EasyMock.createNiceMock(SeekableStreamSupervisorIOConfig.class); + + Map existingSpecs = ImmutableMap.of("id1", streamSpec); + + // Setup expectations for supervisor startup + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); + EasyMock.expect(streamSpec.getId()).andReturn("id1").anyTimes(); + EasyMock.expect(streamSpec.getDataSources()).andReturn(ImmutableList.of("datasource")).anyTimes(); + EasyMock.expect(streamSpec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(streamSpec.createSupervisor()).andReturn(streamSupervisor).anyTimes(); + EasyMock.expect(streamSpec.createAutoscaler(streamSupervisor)).andReturn(null).anyTimes(); + EasyMock.expect(streamSpec.getContext()).andReturn(ImmutableMap.of("useConcurrentLocks", true)).anyTimes(); + + // Expectations for resetSupervisorAndBackfill + EasyMock.expect(streamSupervisor.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(ioConfig.isUseEarliestSequenceNumber()).andReturn(false); + EasyMock.expect(streamSupervisor.getState()).andReturn(SupervisorStateManager.BasicState.RUNNING).anyTimes(); + + streamSupervisor.updatePartitionLagFromStream(); + EasyMock.expect(streamSupervisor.getLatestSequencesFromStream()) + .andReturn(ImmutableMap.of("0", 150L, "1", 250L)); + + EasyMock.expect(streamSupervisor.getOffsetsFromMetadataStorage()) + .andReturn(ImmutableMap.of("0", 100L, "1", 200L)); + + EasyMock.expect(ioConfig.getStream()).andReturn("test-stream"); + EasyMock.expect(streamSupervisor.createDataSourceMetaDataForReset( + EasyMock.eq("test-stream"), + EasyMock.anyObject() + )).andReturn(new TestSeekableStreamDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>("test-stream", ImmutableMap.of("0", "150", "1", "250"), ImmutableSet.of()) + )); + + streamSupervisor.resetOffsets(EasyMock.anyObject(DataSourceMetadata.class)); + streamSupervisor.submitBackfillTask( + EasyMock.anyObject(), + EasyMock.anyObject() + ); + + replayAll(); + EasyMock.replay(streamSupervisor, streamSpec, ioConfig); + + manager.start(); + + Map result = manager.resetSupervisorAndBackfill("id1"); + + Assert.assertEquals("id1", result.get("id")); + Assert.assertNotNull(result.get("backfillRange")); + + Map backfillRange = (Map) result.get("backfillRange"); + Assert.assertEquals(2, backfillRange.size()); + Assert.assertEquals(ImmutableMap.of("start", 100L, "end", 150L), backfillRange.get("0")); + Assert.assertEquals(ImmutableMap.of("start", 200L, "end", 250L), backfillRange.get("1")); + + verifyAll(); + EasyMock.verify(streamSupervisor, streamSpec, ioConfig); + } + + @Test + public void testResetSupervisorAndBackfillWithNonExistentSupervisor() + { + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(ImmutableMap.of()); + replayAll(); + + manager.start(); + + IllegalArgumentException exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> manager.resetSupervisorAndBackfill("nonexistent") + ); + Assert.assertEquals("Supervisor[nonexistent] does not exist", exception.getMessage()); + + verifyAll(); + } + + @Test + public void testResetSupervisorAndBackfillWithNonStreamSupervisor() + { + Map existingSpecs = ImmutableMap.of( + "id3", new TestSupervisorSpec("id3", supervisor3) + ); + + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); + supervisor3.start(); + EasyMock.expect(supervisor3.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); + replayAll(); + + manager.start(); + + IllegalArgumentException exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> manager.resetSupervisorAndBackfill("id3") + ); + Assert.assertEquals("Supervisor[id3] is not a SeekableStreamSupervisor", exception.getMessage()); + + verifyAll(); + } + + @Test + public void testResetSupervisorAndBackfillWithUseConcurrentLocksDisabled() + { + SeekableStreamSupervisor streamSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); + SeekableStreamSupervisorSpec streamSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); + SeekableStreamSupervisorIOConfig ioConfig = EasyMock.createNiceMock(SeekableStreamSupervisorIOConfig.class); + + Map existingSpecs = ImmutableMap.of("id1", streamSpec); + + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); + EasyMock.expect(streamSpec.getId()).andReturn("id1").anyTimes(); + EasyMock.expect(streamSpec.getDataSources()).andReturn(ImmutableList.of("datasource")).anyTimes(); + EasyMock.expect(streamSpec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(streamSpec.createSupervisor()).andReturn(streamSupervisor).anyTimes(); + EasyMock.expect(streamSpec.createAutoscaler(streamSupervisor)).andReturn(null).anyTimes(); + // useConcurrentLocks is false + EasyMock.expect(streamSpec.getContext()).andReturn(ImmutableMap.of("useConcurrentLocks", false)).anyTimes(); + + EasyMock.expect(streamSupervisor.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(ioConfig.isUseEarliestSequenceNumber()).andReturn(false); + + replayAll(); + EasyMock.replay(streamSupervisor, streamSpec, ioConfig); + + manager.start(); + + IllegalArgumentException exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> manager.resetSupervisorAndBackfill("id1") + ); + Assert.assertTrue( + "Expected error message about useConcurrentLocks", + exception.getMessage().contains("Backfill tasks require 'useConcurrentLocks'") + ); + + verifyAll(); + EasyMock.verify(streamSupervisor, streamSpec, ioConfig); + } + + @Test + public void testResetSupervisorAndBackfillWithUseEarliestOffsetEnabled() + { + SeekableStreamSupervisor streamSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); + SeekableStreamSupervisorSpec streamSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); + SeekableStreamSupervisorIOConfig ioConfig = EasyMock.createNiceMock(SeekableStreamSupervisorIOConfig.class); + + Map existingSpecs = ImmutableMap.of("id1", streamSpec); + + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); + EasyMock.expect(streamSpec.getId()).andReturn("id1").anyTimes(); + EasyMock.expect(streamSpec.getDataSources()).andReturn(ImmutableList.of("datasource")).anyTimes(); + EasyMock.expect(streamSpec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(streamSpec.createSupervisor()).andReturn(streamSupervisor).anyTimes(); + EasyMock.expect(streamSpec.createAutoscaler(streamSupervisor)).andReturn(null).anyTimes(); + EasyMock.expect(streamSpec.getContext()).andReturn(ImmutableMap.of("useConcurrentLocks", true)).anyTimes(); + + EasyMock.expect(streamSupervisor.getIoConfig()).andReturn(ioConfig).anyTimes(); + // useEarliestOffset is true + EasyMock.expect(ioConfig.isUseEarliestSequenceNumber()).andReturn(true); + + replayAll(); + EasyMock.replay(streamSupervisor, streamSpec, ioConfig); + + manager.start(); + + IllegalArgumentException exception = Assert.assertThrows( + IllegalArgumentException.class, + () -> manager.resetSupervisorAndBackfill("id1") + ); + Assert.assertTrue( + "Expected error message about useEarliestOffset", + exception.getMessage().contains("Reset with skipped offsets is not supported when useEarliestOffset is true") + ); + + verifyAll(); + EasyMock.verify(streamSupervisor, streamSpec, ioConfig); + } + + @Test + public void testResetSupervisorAndBackfillWithNonRunningSupervisor() + { + SeekableStreamSupervisor streamSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); + SeekableStreamSupervisorSpec streamSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); + SeekableStreamSupervisorIOConfig ioConfig = EasyMock.createNiceMock(SeekableStreamSupervisorIOConfig.class); + + Map existingSpecs = ImmutableMap.of("id1", streamSpec); + + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); + EasyMock.expect(streamSpec.getId()).andReturn("id1").anyTimes(); + EasyMock.expect(streamSpec.getDataSources()).andReturn(ImmutableList.of("datasource")).anyTimes(); + EasyMock.expect(streamSpec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(streamSpec.createSupervisor()).andReturn(streamSupervisor).anyTimes(); + EasyMock.expect(streamSpec.createAutoscaler(streamSupervisor)).andReturn(null).anyTimes(); + EasyMock.expect(streamSpec.getContext()).andReturn(ImmutableMap.of("useConcurrentLocks", true)).anyTimes(); + + EasyMock.expect(streamSupervisor.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(ioConfig.isUseEarliestSequenceNumber()).andReturn(false); + // Supervisor is IDLE, not RUNNING + EasyMock.expect(streamSupervisor.getState()).andReturn(SupervisorStateManager.BasicState.IDLE).anyTimes(); + + replayAll(); + EasyMock.replay(streamSupervisor, streamSpec, ioConfig); + + manager.start(); + + IllegalStateException exception = Assert.assertThrows( + IllegalStateException.class, + () -> manager.resetSupervisorAndBackfill("id1") + ); + Assert.assertTrue( + "Expected error message about running supervisor", + exception.getMessage().contains("A running supervisor is required") + ); + + verifyAll(); + EasyMock.verify(streamSupervisor, streamSpec, ioConfig); + } + @Test public void testCreateSuspendResumeAndStopSupervisor() { From fbb351657fc8e6d81655e24d4ffad1d4a844575e Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 27 Mar 2026 12:35:13 -0700 Subject: [PATCH 21/27] Increase code coverage for SupervisorResourceTest --- .../supervisor/SupervisorResourceTest.java | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index a3d13f718671..307df4dff2a8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -1340,6 +1340,7 @@ public void testResetOffsetsAndBackfill() ) ); + // Test 200 - Success EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); EasyMock.expect(supervisorManager.resetSupervisorAndBackfill( EasyMock.capture(id1) @@ -1355,6 +1356,71 @@ public void testResetOffsetsAndBackfill() resetAll(); + // Test 400 - Supervisor not found + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("non-existent")) + .andThrow(new IllegalArgumentException("Supervisor[non-existent] does not exist")); + replayAll(); + + response = supervisorResource.reset("non-existent", true); + + Assert.assertEquals(400, response.getStatus()); + Assert.assertTrue( + response.getEntity().toString().contains("Supervisor[non-existent] does not exist") + ); + verifyAll(); + + resetAll(); + + // Test 400 - useConcurrentLocks not enabled + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id")) + .andThrow(new IllegalArgumentException("Backfill tasks require 'useConcurrentLocks' to be set to true")); + replayAll(); + + response = supervisorResource.reset("my-id", true); + + Assert.assertEquals(400, response.getStatus()); + Assert.assertTrue( + response.getEntity().toString().contains("Backfill tasks require 'useConcurrentLocks'") + ); + verifyAll(); + + resetAll(); + + // Test 400 - useEarliestOffset is enabled + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id")) + .andThrow(new IllegalArgumentException("Reset with skipped offsets is not supported when useEarliestOffset is true")); + replayAll(); + + response = supervisorResource.reset("my-id", true); + + Assert.assertEquals(400, response.getStatus()); + Assert.assertTrue( + response.getEntity().toString().contains("Reset with skipped offsets is not supported when useEarliestOffset is true") + ); + verifyAll(); + + resetAll(); + + // Test 404 - Supervisor not running + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id")) + .andThrow(new IllegalStateException("A running supervisor is required to query the latest offsets from the stream")); + replayAll(); + + response = supervisorResource.reset("my-id", true); + + Assert.assertEquals(404, response.getStatus()); + Assert.assertTrue( + response.getEntity().toString().contains("A running supervisor is required") + ); + verifyAll(); + + resetAll(); + + // Test 503 - Service unavailable EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()); replayAll(); From 90f9d076edc74c2c53d7478613653b3a3144ac76 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 27 Mar 2026 13:40:54 -0700 Subject: [PATCH 22/27] Unit tests for KafkaSupervisor submitBackfillTask --- .../kafka/supervisor/KafkaSupervisorTest.java | 138 ++++++++++++++++++ 1 file changed, 138 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 9e11053939d9..00c63ccea0fd 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -6207,6 +6207,144 @@ public int getPendingCompletionTaskGroupsCount(int groupId) } } + @Test + public void testSubmitBackfillTask() + { + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(null).anyTimes(); + + Capture capturedTasks = Capture.newInstance(CaptureType.ALL); + EasyMock.expect(taskQueue.add(EasyMock.capture(capturedTasks))).andReturn(true).times(2); + + replayAll(); + + supervisor = getTestableSupervisor(2, 4, true, false, null, null, null); + + // Create start and end offsets for 3 partitions + Map startOffsets = ImmutableMap.of( + new KafkaTopicPartition(false, topic, 0), 100L, + new KafkaTopicPartition(false, topic, 1), 200L, + new KafkaTopicPartition(false, topic, 2), 300L + ); + + Map endOffsets = ImmutableMap.of( + new KafkaTopicPartition(false, topic, 0), 150L, + new KafkaTopicPartition(false, topic, 1), 250L, + new KafkaTopicPartition(false, topic, 2), 350L + ); + + supervisor.submitBackfillTask(startOffsets, endOffsets); + + List tasks = capturedTasks.getValues(); + Assert.assertEquals(2, tasks.size()); + + // Verify both tasks are KafkaIndexTask + for (Task task : tasks) { + Assert.assertTrue(task instanceof KafkaIndexTask); + KafkaIndexTask kafkaTask = (KafkaIndexTask) task; + + // Verify useTransaction=false for backfill tasks + Assert.assertFalse( + "Backfill tasks should have useTransaction=false", + kafkaTask.getIOConfig().isUseTransaction() + ); + + // Verify task has correct datasource + Assert.assertEquals(DATASOURCE, kafkaTask.getDataSource()); + + // Verify offsets are within the expected range + Map taskStartOffsets = + kafkaTask.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap(); + Map taskEndOffsets = + kafkaTask.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap(); + + for (Map.Entry entry : taskStartOffsets.entrySet()) { + KafkaTopicPartition partition = entry.getKey(); + Long startOffset = entry.getValue(); + Long endOffset = taskEndOffsets.get(partition); + + // Verify offsets are from our original maps + Assert.assertTrue(startOffsets.containsKey(partition)); + Assert.assertEquals(startOffsets.get(partition), startOffset); + Assert.assertEquals(endOffsets.get(partition), endOffset); + } + } + + verifyAll(); + } + + @Test + public void testSubmitBackfillTaskWithNullStartOffset() + { + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(null).anyTimes(); + + Capture capturedTask = Capture.newInstance(); + EasyMock.expect(taskQueue.add(EasyMock.capture(capturedTask))).andReturn(true).once(); + + replayAll(); + + supervisor = getTestableSupervisor(2, 2, true, false, null, null, null); + + KafkaTopicPartition partition0 = new KafkaTopicPartition(false, topic, 0); + KafkaTopicPartition partition1 = new KafkaTopicPartition(false, topic, 1); + + // partition0 has a start offset, partition1 does not (null in startOffsets map) + Map startOffsets = ImmutableMap.of( + partition0, 100L + // partition1 intentionally missing - simulates no checkpoint for this partition + ); + + Map endOffsets = ImmutableMap.of( + partition0, 150L, + partition1, 250L + ); + + supervisor.submitBackfillTask(startOffsets, endOffsets); + + // Verify task was submitted + Task task = capturedTask.getValue(); + Assert.assertTrue(task instanceof KafkaIndexTask); + KafkaIndexTask kafkaTask = (KafkaIndexTask) task; + + Map taskStartOffsets = + kafkaTask.getIOConfig().getStartSequenceNumbers().getPartitionSequenceNumberMap(); + Map taskEndOffsets = + kafkaTask.getIOConfig().getEndSequenceNumbers().getPartitionSequenceNumberMap(); + + // partition0 should use its start offset + Assert.assertEquals(Long.valueOf(100L), taskStartOffsets.get(partition0)); + Assert.assertEquals(Long.valueOf(150L), taskEndOffsets.get(partition0)); + + // partition1 should have startOffset set equal to endOffset (since start was null) + Assert.assertEquals(Long.valueOf(250L), taskStartOffsets.get(partition1)); + Assert.assertEquals(Long.valueOf(250L), taskEndOffsets.get(partition1)); + + verifyAll(); + } + + @Test + public void testSubmitBackfillTaskWithEmptyOffsets() + { + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); + + replayAll(); + + supervisor = getTestableSupervisor(2, 2, true, false, null, null, null); + + // Submit with empty offsets - should return early without submitting any tasks + supervisor.submitBackfillTask(ImmutableMap.of(), ImmutableMap.of()); + + // Verify no tasks were submitted (taskQueue.add should never be called) + verifyAll(); + } + private static class TestableKafkaSupervisorWithCustomIsTaskCurrent extends TestableKafkaSupervisor { private final boolean isTaskCurrentReturn; From c368e6a29b013483ea9ac1759a37cf4227c253a0 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 27 Mar 2026 14:31:44 -0700 Subject: [PATCH 23/27] Fail early and don't reset it earliest/latest offsets are empty --- .../supervisor/SupervisorManager.java | 8 ++ .../supervisor/SupervisorManagerTest.java | 89 +++++++++++++++++++ 2 files changed, 97 insertions(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 1b7aecea27ee..9fb6644f3be3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -451,6 +451,14 @@ public Map resetSupervisorAndBackfill(String id) log.info("Capturing checkpointed offsets for supervisor[%s]", id); Map startOffsets = streamSupervisor.getOffsetsFromMetadataStorage(); + // Validate that we successfully retrieved offsets + if (latestOffsets == null || latestOffsets.isEmpty()) { + throw new ISE("Skipping reset: Failed to get latest offsets from stream for supervisor[%s]", id); + } + if (startOffsets == null || startOffsets.isEmpty()) { + throw new ISE("Skipping reset: Failed to get checkpointed offsets for supervisor[%s]", id); + } + log.info("Resetting supervisor[%s] metadata to latest offsets", id); DataSourceMetadata resetMetadata = streamSupervisor.createDataSourceMetaDataForReset( streamSupervisor.getIoConfig().getStream(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index a8f8023906f6..08613b326061 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -907,6 +907,95 @@ public void testResetSupervisorAndBackfillWithNonRunningSupervisor() EasyMock.verify(streamSupervisor, streamSpec, ioConfig); } + @Test + public void testResetSupervisorAndBackfillWithEmptyLatestOffsets() + { + SeekableStreamSupervisor streamSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); + SeekableStreamSupervisorSpec streamSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); + SeekableStreamSupervisorIOConfig ioConfig = EasyMock.createNiceMock(SeekableStreamSupervisorIOConfig.class); + + Map existingSpecs = ImmutableMap.of("id1", streamSpec); + + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); + EasyMock.expect(streamSpec.getId()).andReturn("id1").anyTimes(); + EasyMock.expect(streamSpec.getDataSources()).andReturn(ImmutableList.of("datasource")).anyTimes(); + EasyMock.expect(streamSpec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(streamSpec.createSupervisor()).andReturn(streamSupervisor).anyTimes(); + EasyMock.expect(streamSpec.createAutoscaler(streamSupervisor)).andReturn(null).anyTimes(); + EasyMock.expect(streamSpec.getContext()).andReturn(ImmutableMap.of("useConcurrentLocks", true)).anyTimes(); + + EasyMock.expect(streamSupervisor.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(ioConfig.isUseEarliestSequenceNumber()).andReturn(false); + EasyMock.expect(streamSupervisor.getState()).andReturn(SupervisorStateManager.BasicState.RUNNING).anyTimes(); + + // Mock empty latest offsets + streamSupervisor.updatePartitionLagFromStream(); + EasyMock.expectLastCall(); + EasyMock.expect(streamSupervisor.getLatestSequencesFromStream()).andReturn(ImmutableMap.of()); + + replayAll(); + EasyMock.replay(streamSupervisor, streamSpec, ioConfig); + + manager.start(); + + IllegalStateException exception = Assert.assertThrows( + IllegalStateException.class, + () -> manager.resetSupervisorAndBackfill("id1") + ); + Assert.assertTrue( + "Expected error message about failing to get latest offsets", + exception.getMessage().contains("Skipping reset: Failed to get latest offsets from stream for supervisor") + ); + + verifyAll(); + EasyMock.verify(streamSupervisor, streamSpec, ioConfig); + } + + @Test + public void testResetSupervisorAndBackfillWithEmptyStartOffsets() + { + SeekableStreamSupervisor streamSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); + SeekableStreamSupervisorSpec streamSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); + SeekableStreamSupervisorIOConfig ioConfig = EasyMock.createNiceMock(SeekableStreamSupervisorIOConfig.class); + + Map existingSpecs = ImmutableMap.of("id1", streamSpec); + + EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); + EasyMock.expect(streamSpec.getId()).andReturn("id1").anyTimes(); + EasyMock.expect(streamSpec.getDataSources()).andReturn(ImmutableList.of("datasource")).anyTimes(); + EasyMock.expect(streamSpec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(streamSpec.createSupervisor()).andReturn(streamSupervisor).anyTimes(); + EasyMock.expect(streamSpec.createAutoscaler(streamSupervisor)).andReturn(null).anyTimes(); + EasyMock.expect(streamSpec.getContext()).andReturn(ImmutableMap.of("useConcurrentLocks", true)).anyTimes(); + + EasyMock.expect(streamSupervisor.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(ioConfig.isUseEarliestSequenceNumber()).andReturn(false); + EasyMock.expect(streamSupervisor.getState()).andReturn(SupervisorStateManager.BasicState.RUNNING).anyTimes(); + + // Mock non-empty latest offsets but empty start offsets + streamSupervisor.updatePartitionLagFromStream(); + EasyMock.expectLastCall(); + EasyMock.expect(streamSupervisor.getLatestSequencesFromStream()).andReturn(ImmutableMap.of("0", 100L)); + EasyMock.expect(streamSupervisor.getOffsetsFromMetadataStorage()).andReturn(ImmutableMap.of()); + + replayAll(); + EasyMock.replay(streamSupervisor, streamSpec, ioConfig); + + manager.start(); + + IllegalStateException exception = Assert.assertThrows( + IllegalStateException.class, + () -> manager.resetSupervisorAndBackfill("id1") + ); + Assert.assertTrue( + "Expected error message about failing to get checkpointed offsets", + exception.getMessage().contains("Skipping reset: Failed to get checkpointed offsets for supervisor") + ); + + verifyAll(); + EasyMock.verify(streamSupervisor, streamSpec, ioConfig); + } + @Test public void testCreateSuspendResumeAndStopSupervisor() { From 66705c05952ac48b67ef9c7ef2651ac4ed026daf Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 27 Mar 2026 14:37:10 -0700 Subject: [PATCH 24/27] Update SupervisorManager.java --- .../druid/indexing/overlord/supervisor/SupervisorManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 9fb6644f3be3..ac54312a5efd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -410,7 +410,7 @@ public boolean resetSupervisor(String id, @Nullable DataSourceMetadata resetData * @param id supervisor ID * @return Map containing supervisorId and skipped offset ranges * @throws IllegalArgumentException if supervisor doesn't exist or if useEarliestOffset is true - * @throws IllegalStateException if supervisor is not running + * @throws IllegalStateException if supervisor is not running or if either checkpointed or latest offsets is empty */ public Map resetSupervisorAndBackfill(String id) { From fe7e6bb88d158b9f8fd916d5ab71ad099384a69a Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 10 Apr 2026 10:29:04 -0700 Subject: [PATCH 25/27] Refactor into separate resetOffsetsAndBackfill endpoint --- .../supervisor/SupervisorResource.java | 20 ++++++++++--------- .../supervisor/SupervisorResourceTest.java | 16 +++++++-------- 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index 219c09b1aba9..1dfd1dd0fe01 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -592,16 +592,18 @@ public Response specGetHistory( @Path("/{id}/reset") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(SupervisorResourceFilter.class) - public Response reset( - @PathParam("id") final String id, - @QueryParam("backfill") Boolean backfill - ) + public Response reset(@PathParam("id") final String id) { - if (Boolean.TRUE.equals(backfill)) { - return handleResetAndBackfill(id); - } else { - return handleResetRequest(id, null); - } + return handleResetRequest(id, null); + } + + @POST + @Path("/{id}/resetOffsetsAndBackfill") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(SupervisorResourceFilter.class) + public Response resetOffsetsAndBackfill(@PathParam("id") final String id) + { + return handleResetAndBackfill(id); } @POST diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 307df4dff2a8..eaa52951bad9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -1257,12 +1257,12 @@ public void testReset() )).andReturn(false); replayAll(); - Response response = supervisorResource.reset("my-id", false); + Response response = supervisorResource.reset("my-id"); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(ImmutableMap.of("id", "my-id"), response.getEntity()); - response = supervisorResource.reset("my-id-2", false); + response = supervisorResource.reset("my-id-2"); Assert.assertEquals(404, response.getStatus()); Assert.assertEquals("my-id", id1.getValue()); @@ -1347,7 +1347,7 @@ public void testResetOffsetsAndBackfill() )).andReturn(expectedResult); replayAll(); - Response response = supervisorResource.reset("my-id", true); + Response response = supervisorResource.resetOffsetsAndBackfill("my-id"); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(expectedResult, response.getEntity()); @@ -1362,7 +1362,7 @@ public void testResetOffsetsAndBackfill() .andThrow(new IllegalArgumentException("Supervisor[non-existent] does not exist")); replayAll(); - response = supervisorResource.reset("non-existent", true); + response = supervisorResource.resetOffsetsAndBackfill("non-existent"); Assert.assertEquals(400, response.getStatus()); Assert.assertTrue( @@ -1378,7 +1378,7 @@ public void testResetOffsetsAndBackfill() .andThrow(new IllegalArgumentException("Backfill tasks require 'useConcurrentLocks' to be set to true")); replayAll(); - response = supervisorResource.reset("my-id", true); + response = supervisorResource.resetOffsetsAndBackfill("my-id"); Assert.assertEquals(400, response.getStatus()); Assert.assertTrue( @@ -1394,7 +1394,7 @@ public void testResetOffsetsAndBackfill() .andThrow(new IllegalArgumentException("Reset with skipped offsets is not supported when useEarliestOffset is true")); replayAll(); - response = supervisorResource.reset("my-id", true); + response = supervisorResource.resetOffsetsAndBackfill("my-id"); Assert.assertEquals(400, response.getStatus()); Assert.assertTrue( @@ -1410,7 +1410,7 @@ public void testResetOffsetsAndBackfill() .andThrow(new IllegalStateException("A running supervisor is required to query the latest offsets from the stream")); replayAll(); - response = supervisorResource.reset("my-id", true); + response = supervisorResource.resetOffsetsAndBackfill("my-id"); Assert.assertEquals(404, response.getStatus()); Assert.assertTrue( @@ -1424,7 +1424,7 @@ public void testResetOffsetsAndBackfill() EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()); replayAll(); - response = supervisorResource.reset("my-id", true); + response = supervisorResource.resetOffsetsAndBackfill("my-id"); Assert.assertEquals(503, response.getStatus()); verifyAll(); From 4e9b68c8521b048319b7efbc568d78afe603fe93 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 10 Apr 2026 13:13:11 -0700 Subject: [PATCH 26/27] Set backfillTaskCount through API param instead of in IOConfig --- .../RabbitStreamSupervisorIOConfig.java | 3 +-- .../kafka/supervisor/KafkaSupervisor.java | 10 +++++---- .../supervisor/KafkaSupervisorIOConfig.java | 6 ++--- .../supervisor/KafkaIOConfigBuilder.java | 1 - .../KafkaSupervisorIOConfigTest.java | 3 --- .../kafka/supervisor/KafkaSupervisorTest.java | 14 +++++------- .../supervisor/KinesisSupervisorIOConfig.java | 3 +-- .../supervisor/SupervisorManager.java | 4 ++-- .../supervisor/SupervisorResource.java | 11 ++++++---- .../supervisor/SeekableStreamSupervisor.java | 4 +++- .../SeekableStreamSupervisorIOConfig.java | 13 +---------- .../supervisor/SupervisorManagerTest.java | 19 ++++++++-------- .../supervisor/SupervisorResourceTest.java | 22 +++++++++---------- .../SeekableStreamSamplerSpecTest.java | 1 - .../SeekableStreamSupervisorIOConfigTest.java | 13 +---------- .../SeekableStreamSupervisorSpecTest.java | 4 ---- .../SeekableStreamSupervisorStateTest.java | 10 +-------- .../SeekableStreamSupervisorTestBase.java | 1 - 18 files changed, 51 insertions(+), 91 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java index 51a4c340658d..8aad5b762219 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java @@ -86,8 +86,7 @@ public RabbitStreamSupervisorIOConfig( lateMessageRejectionStartDateTime, new IdleConfig(null, null), stopTaskCount, - serverPriorityToReplicas, - null + serverPriorityToReplicas ); this.consumerProperties = consumerProperties; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index bc79a404e9b2..ea1aa0caa151 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -270,7 +270,8 @@ protected List startOffsets, - Map endOffsets + Map endOffsets, + @Nullable Integer backfillTaskCount ) { if (startOffsets == null || startOffsets.isEmpty() || endOffsets == null || endOffsets.isEmpty()) { @@ -281,12 +282,13 @@ public void submitBackfillTask( try { String backfillSupervisorId = spec.getSpec().getDataSchema().getDataSource() + "_backfill"; - // Get the backfillTaskCount from config - int backfillTaskCount = spec.getSpec().getIOConfig().getBackfillTaskCount(); + // If backfillTaskCount is not provided, default to taskCount / 2 + int taskCount = spec.getSpec().getIOConfig().getTaskCount(); + int numBackfillTasks = backfillTaskCount != null ? backfillTaskCount : Math.max(1, taskCount / 2); List partitions = new ArrayList<>(endOffsets.keySet()); // Determine actual number of tasks (can't have more tasks than partitions) - int numTasks = Math.min(backfillTaskCount, partitions.size()); + int numTasks = Math.min(numBackfillTasks, partitions.size()); log.info( "Submitting %d backfill task(s) with supervisorId[%s] for %d partition(s)", diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index 1f144e49638a..992ff292694a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -78,8 +78,7 @@ public KafkaSupervisorIOConfig( @JsonProperty("idleConfig") IdleConfig idleConfig, @JsonProperty("stopTaskCount") Integer stopTaskCount, @Nullable @JsonProperty("emitTimeLagMetrics") Boolean emitTimeLagMetrics, - @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas, - @Nullable @JsonProperty("backfillTaskCount") Integer backfillTaskCount + @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas ) { super( @@ -99,8 +98,7 @@ public KafkaSupervisorIOConfig( lateMessageRejectionStartDateTime, idleConfig, stopTaskCount, - serverPriorityToReplicas, - backfillTaskCount + serverPriorityToReplicas ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java index dd62c8385f45..24c1656fc7e6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaIOConfigBuilder.java @@ -93,7 +93,6 @@ public KafkaSupervisorIOConfig build() idleConfig, stopTaskCount, null, - null, null ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java index 049b9959088c..6295d41937e8 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java @@ -341,7 +341,6 @@ public void testAutoScalingConfigSerde() throws JsonProcessingException null, null, false, - null, null ); String ioConfig = mapper.writeValueAsString(kafkaSupervisorIOConfig); @@ -378,7 +377,6 @@ public void testAutoScalingConfigSerde() throws JsonProcessingException null, null, false, - null, null ); Assert.assertEquals(5, kafkaSupervisorIOConfig.getTaskCount().intValue()); @@ -432,7 +430,6 @@ public void testIdleConfigSerde() throws JsonProcessingException mapper.convertValue(idleConfig, IdleConfig.class), null, false, - null, null ); String ioConfig = mapper.writeValueAsString(kafkaSupervisorIOConfig); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 00c63ccea0fd..c085d5f0b2e9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -5401,8 +5401,7 @@ public void testComputeUnassignedServerPriorities_whenMultipleReplicasPerPriorit Map.of( 10, 2, 20, 3 - ), - null + ) ); Assert.assertEquals(5, (int) kafkaSupervisorIOConfig.getReplicas()); @@ -5687,8 +5686,7 @@ private TestableKafkaSupervisor getTestableSupervisor( idleConfig, null, true, - serverPriorityToReplicas, - null + serverPriorityToReplicas ); KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( @@ -5783,7 +5781,6 @@ private TestableKafkaSupervisor getTestableSupervisorCustomIsTaskCurrent( null, null, false, - null, null ); @@ -5879,7 +5876,6 @@ private KafkaSupervisor createSupervisor( null, null, false, - null, null ); @@ -6235,7 +6231,7 @@ public void testSubmitBackfillTask() new KafkaTopicPartition(false, topic, 2), 350L ); - supervisor.submitBackfillTask(startOffsets, endOffsets); + supervisor.submitBackfillTask(startOffsets, endOffsets, null); List tasks = capturedTasks.getValues(); Assert.assertEquals(2, tasks.size()); @@ -6304,7 +6300,7 @@ public void testSubmitBackfillTaskWithNullStartOffset() partition1, 250L ); - supervisor.submitBackfillTask(startOffsets, endOffsets); + supervisor.submitBackfillTask(startOffsets, endOffsets, null); // Verify task was submitted Task task = capturedTask.getValue(); @@ -6339,7 +6335,7 @@ public void testSubmitBackfillTaskWithEmptyOffsets() supervisor = getTestableSupervisor(2, 2, true, false, null, null, null); // Submit with empty offsets - should return early without submitting any tasks - supervisor.submitBackfillTask(ImmutableMap.of(), ImmutableMap.of()); + supervisor.submitBackfillTask(ImmutableMap.of(), ImmutableMap.of(), null); // Verify no tasks were submitted (taskQueue.add should never be called) verifyAll(); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index e1770f42b57d..6c325bd0744d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -99,8 +99,7 @@ public KinesisSupervisorIOConfig( lateMessageRejectionStartDateTime, new IdleConfig(null, null), null, - serverPriorityToReplicas, - null + serverPriorityToReplicas ); this.endpoint = endpoint != null diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index ac54312a5efd..81e361bd057f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -412,7 +412,7 @@ public boolean resetSupervisor(String id, @Nullable DataSourceMetadata resetData * @throws IllegalArgumentException if supervisor doesn't exist or if useEarliestOffset is true * @throws IllegalStateException if supervisor is not running or if either checkpointed or latest offsets is empty */ - public Map resetSupervisorAndBackfill(String id) + public Map resetSupervisorAndBackfill(String id, @Nullable Integer backfillTaskCount) { Preconditions.checkState(started, "SupervisorManager not started"); Preconditions.checkNotNull(id, "id"); @@ -475,7 +475,7 @@ public Map resetSupervisorAndBackfill(String id) Map backfillRange = calculateBackfillRange(startOffsets, latestOffsets); - streamSupervisor.submitBackfillTask(startOffsets, latestOffsets); + streamSupervisor.submitBackfillTask(startOffsets, latestOffsets, backfillTaskCount); log.info("Successfully reset supervisor[%s] to latest. Backfill range: %s", id, backfillRange); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index 1dfd1dd0fe01..10965dffc427 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -601,9 +601,12 @@ public Response reset(@PathParam("id") final String id) @Path("/{id}/resetOffsetsAndBackfill") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(SupervisorResourceFilter.class) - public Response resetOffsetsAndBackfill(@PathParam("id") final String id) + public Response resetOffsetsAndBackfill( + @PathParam("id") final String id, + @QueryParam("backfillTaskCount") @Nullable final Integer backfillTaskCount + ) { - return handleResetAndBackfill(id); + return handleResetAndBackfill(id, backfillTaskCount); } @POST @@ -637,12 +640,12 @@ private Response handleResetRequest( ); } - private Response handleResetAndBackfill(final String id) + private Response handleResetAndBackfill(final String id, @Nullable final Integer backfillTaskCount) { return asLeaderWithSupervisorManager( manager -> { try { - Map result = manager.resetSupervisorAndBackfill(id); + Map result = manager.resetSupervisorAndBackfill(id, backfillTaskCount); return Response.ok(result).build(); } catch (IllegalArgumentException e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index bf7f1dc6974a..605d0d0b310f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -4743,10 +4743,12 @@ protected abstract List startOffsets, - Map endOffsets + Map endOffsets, + @Nullable Integer backfillTaskCount ) { log.info("submitBackfillTask not implemented for supervisor[%s], skipping backfill submission", supervisorId); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index 31f1781bb352..633bd9b70dc9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -54,7 +54,6 @@ public abstract class SeekableStreamSupervisorIOConfig @Nullable private final IdleConfig idleConfig; @Nullable private final Integer stopTaskCount; @Nullable private final Map serverPriorityToReplicas; - @Nullable private final Integer backfillTaskCount; private final LagAggregator lagAggregator; private final boolean autoScalerEnabled; @@ -76,8 +75,7 @@ public SeekableStreamSupervisorIOConfig( DateTime lateMessageRejectionStartDateTime, @Nullable IdleConfig idleConfig, @Nullable Integer stopTaskCount, - @Nullable Map serverPriorityToReplicas, - @Nullable Integer backfillTaskCount + @Nullable Map serverPriorityToReplicas ) { this.stream = Preconditions.checkNotNull(stream, "stream cannot be null"); @@ -156,9 +154,6 @@ public SeekableStreamSupervisorIOConfig( } else { this.replicas = replicas != null ? replicas : 1; } - - // Default backfillTaskCount to taskCount / 2 - this.backfillTaskCount = backfillTaskCount != null ? backfillTaskCount : Math.max(1, this.taskCount / 2); } private static Duration defaultDuration(final Period period, final String theDefault) @@ -264,12 +259,6 @@ public Optional getLateMessageRejectionStartDateTime() return lateMessageRejectionStartDateTime; } - @JsonProperty - public Integer getBackfillTaskCount() - { - return backfillTaskCount; - } - @Nullable @JsonProperty public IdleConfig getIdleConfig() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index 08613b326061..2fb30ed33d81 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -727,7 +727,8 @@ public void testResetSupervisorAndBackfill() streamSupervisor.resetOffsets(EasyMock.anyObject(DataSourceMetadata.class)); streamSupervisor.submitBackfillTask( EasyMock.anyObject(), - EasyMock.anyObject() + EasyMock.anyObject(), + null ); replayAll(); @@ -735,7 +736,7 @@ public void testResetSupervisorAndBackfill() manager.start(); - Map result = manager.resetSupervisorAndBackfill("id1"); + Map result = manager.resetSupervisorAndBackfill("id1", null); Assert.assertEquals("id1", result.get("id")); Assert.assertNotNull(result.get("backfillRange")); @@ -759,7 +760,7 @@ public void testResetSupervisorAndBackfillWithNonExistentSupervisor() IllegalArgumentException exception = Assert.assertThrows( IllegalArgumentException.class, - () -> manager.resetSupervisorAndBackfill("nonexistent") + () -> manager.resetSupervisorAndBackfill("nonexistent", null) ); Assert.assertEquals("Supervisor[nonexistent] does not exist", exception.getMessage()); @@ -782,7 +783,7 @@ public void testResetSupervisorAndBackfillWithNonStreamSupervisor() IllegalArgumentException exception = Assert.assertThrows( IllegalArgumentException.class, - () -> manager.resetSupervisorAndBackfill("id3") + () -> manager.resetSupervisorAndBackfill("id3", null) ); Assert.assertEquals("Supervisor[id3] is not a SeekableStreamSupervisor", exception.getMessage()); @@ -817,7 +818,7 @@ public void testResetSupervisorAndBackfillWithUseConcurrentLocksDisabled() IllegalArgumentException exception = Assert.assertThrows( IllegalArgumentException.class, - () -> manager.resetSupervisorAndBackfill("id1") + () -> manager.resetSupervisorAndBackfill("id1", null) ); Assert.assertTrue( "Expected error message about useConcurrentLocks", @@ -856,7 +857,7 @@ public void testResetSupervisorAndBackfillWithUseEarliestOffsetEnabled() IllegalArgumentException exception = Assert.assertThrows( IllegalArgumentException.class, - () -> manager.resetSupervisorAndBackfill("id1") + () -> manager.resetSupervisorAndBackfill("id1", null) ); Assert.assertTrue( "Expected error message about useEarliestOffset", @@ -896,7 +897,7 @@ public void testResetSupervisorAndBackfillWithNonRunningSupervisor() IllegalStateException exception = Assert.assertThrows( IllegalStateException.class, - () -> manager.resetSupervisorAndBackfill("id1") + () -> manager.resetSupervisorAndBackfill("id1", null) ); Assert.assertTrue( "Expected error message about running supervisor", @@ -940,7 +941,7 @@ public void testResetSupervisorAndBackfillWithEmptyLatestOffsets() IllegalStateException exception = Assert.assertThrows( IllegalStateException.class, - () -> manager.resetSupervisorAndBackfill("id1") + () -> manager.resetSupervisorAndBackfill("id1", null) ); Assert.assertTrue( "Expected error message about failing to get latest offsets", @@ -985,7 +986,7 @@ public void testResetSupervisorAndBackfillWithEmptyStartOffsets() IllegalStateException exception = Assert.assertThrows( IllegalStateException.class, - () -> manager.resetSupervisorAndBackfill("id1") + () -> manager.resetSupervisorAndBackfill("id1", null) ); Assert.assertTrue( "Expected error message about failing to get checkpointed offsets", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index eaa52951bad9..3e4d64838088 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -1343,11 +1343,11 @@ public void testResetOffsetsAndBackfill() // Test 200 - Success EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); EasyMock.expect(supervisorManager.resetSupervisorAndBackfill( - EasyMock.capture(id1) + EasyMock.capture(id1), null )).andReturn(expectedResult); replayAll(); - Response response = supervisorResource.resetOffsetsAndBackfill("my-id"); + Response response = supervisorResource.resetOffsetsAndBackfill("my-id", null); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(expectedResult, response.getEntity()); @@ -1358,11 +1358,11 @@ public void testResetOffsetsAndBackfill() // Test 400 - Supervisor not found EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); - EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("non-existent")) + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("non-existent", null)) .andThrow(new IllegalArgumentException("Supervisor[non-existent] does not exist")); replayAll(); - response = supervisorResource.resetOffsetsAndBackfill("non-existent"); + response = supervisorResource.resetOffsetsAndBackfill("non-existent", null); Assert.assertEquals(400, response.getStatus()); Assert.assertTrue( @@ -1374,11 +1374,11 @@ public void testResetOffsetsAndBackfill() // Test 400 - useConcurrentLocks not enabled EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); - EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id")) + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id", null)) .andThrow(new IllegalArgumentException("Backfill tasks require 'useConcurrentLocks' to be set to true")); replayAll(); - response = supervisorResource.resetOffsetsAndBackfill("my-id"); + response = supervisorResource.resetOffsetsAndBackfill("my-id", null); Assert.assertEquals(400, response.getStatus()); Assert.assertTrue( @@ -1390,11 +1390,11 @@ public void testResetOffsetsAndBackfill() // Test 400 - useEarliestOffset is enabled EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); - EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id")) + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id", null)) .andThrow(new IllegalArgumentException("Reset with skipped offsets is not supported when useEarliestOffset is true")); replayAll(); - response = supervisorResource.resetOffsetsAndBackfill("my-id"); + response = supervisorResource.resetOffsetsAndBackfill("my-id", null); Assert.assertEquals(400, response.getStatus()); Assert.assertTrue( @@ -1406,11 +1406,11 @@ public void testResetOffsetsAndBackfill() // Test 404 - Supervisor not running EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); - EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id")) + EasyMock.expect(supervisorManager.resetSupervisorAndBackfill("my-id", null)) .andThrow(new IllegalStateException("A running supervisor is required to query the latest offsets from the stream")); replayAll(); - response = supervisorResource.resetOffsetsAndBackfill("my-id"); + response = supervisorResource.resetOffsetsAndBackfill("my-id", null); Assert.assertEquals(404, response.getStatus()); Assert.assertTrue( @@ -1424,7 +1424,7 @@ public void testResetOffsetsAndBackfill() EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()); replayAll(); - response = supervisorResource.resetOffsetsAndBackfill("my-id"); + response = supervisorResource.resetOffsetsAndBackfill("my-id", null); Assert.assertEquals(503, response.getStatus()); verifyAll(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java index 6fd6307e50c7..5af21ca7b6aa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java @@ -333,7 +333,6 @@ private TestableSeekableStreamSupervisorIOConfig( lateMessageRejectionStartDateTime, idleConfig, null, - null, null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java index c0b7dc1d753a..3974b9bebca9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfigTest.java @@ -69,7 +69,6 @@ public void testAllDefaults() null, null, null, - null, null ) { @@ -122,7 +121,6 @@ public void testAutoScalerEnabledPreservesTaskCountWhenNonNull() null, null, null, - null, null ) { @@ -148,7 +146,6 @@ public void testAutoScalerEnabledPreservesTaskCountWhenNonNull() null, null, null, - null, null ) { @@ -181,7 +178,6 @@ public void testBothLateMessageRejectionPeriodAndStartDateTime() DateTimes.nowUtc(), null, null, - null, null ) { @@ -217,7 +213,6 @@ public void testNullAggregatorThrows() null, null, null, - null, null ) { @@ -251,7 +246,6 @@ public void testGetMaxAllowedStopsScalingDisabled() null, null, null, - null, null ) { @@ -276,7 +270,6 @@ public void testGetMaxAllowedStopsScalingDisabled() null, null, 3, - null, null ) { @@ -313,7 +306,6 @@ public void testGetMaxAllowedStopsScalingEnabled() null, null, 1, - null, null ) { @@ -347,7 +339,6 @@ public void testGetMaxAllowedStopsScalingEnabled() null, null, 1, - null, null ) { @@ -378,7 +369,6 @@ public void testGetMaxAllowedStopsScalingEnabled() null, null, null, - null, null ) { @@ -466,8 +456,7 @@ private SeekableStreamSupervisorIOConfig makeSeekableStreamSupervisorIOConfig(@N null, null, null, - serverPriorityToReplicas, - null + serverPriorityToReplicas ) { }; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java index e967bf946a0d..f0bdac00a8ce 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpecTest.java @@ -791,7 +791,6 @@ public void testSeekableStreamSupervisorSpecWithScaleDisable() throws Interrupte null, null, null, - null, null ) { @@ -850,7 +849,6 @@ public void testEnablingIdleBeviourPerSupervisorWithOverlordConfigEnabled() null, new IdleConfig(true, null), null, - null, null ) { @@ -1463,7 +1461,6 @@ private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scal null, null, null, - null, null ) { @@ -1486,7 +1483,6 @@ private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scal null, null, null, - null, null ) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 845b20f1e709..c91016ff2422 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -697,7 +697,6 @@ public void testIdleStateTransition() throws Exception null, new IdleConfig(true, 200L), null, - null, null ) { @@ -806,7 +805,6 @@ public void testIdleOnStartUpAndTurnsToRunningAfterLagUpdates() null, new IdleConfig(true, 200L), null, - null, null ) { @@ -1107,7 +1105,6 @@ public void testCheckpointForActiveTaskGroup() throws InterruptedException, Json null, new IdleConfig(true, 200L), null, - null, null ) {}; @@ -1327,7 +1324,6 @@ public void testEarlyStoppingOfTaskGroupBasedOnStopTaskCount() throws Interrupte null, new IdleConfig(true, 200L), stopTaskCount, - null, null ) { @@ -1564,7 +1560,6 @@ public void testSupervisorStopTaskGroupEarly() throws JsonProcessingException, I null, new IdleConfig(true, 200L), null, - null, null ) { @@ -2720,7 +2715,6 @@ private void expectEmitterSupervisor(boolean suspended) null, null, null, - null, null ) { @@ -2785,7 +2779,6 @@ public void testMaxAllowedStopsWithStopTaskCountRatio() null, null, 1, // ensure this is overridden - null, null ) { @@ -2941,8 +2934,7 @@ private static SeekableStreamSupervisorIOConfig createSupervisorIOConfig( null, null, null, - serverPriorityToReplicas, - null + serverPriorityToReplicas ) { }; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java index 69d3c7406dac..4746fef689f4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTestBase.java @@ -537,7 +537,6 @@ protected SeekableStreamSupervisorIOConfig createIOConfig(int taskCount, CostBas null, null, null, - null, null ) { From 7c4cef946b9229309430a377e823dc7cebdd70b2 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 10 Apr 2026 15:03:09 -0700 Subject: [PATCH 27/27] Unit test fix --- .../indexing/overlord/supervisor/SupervisorManagerTest.java | 2 +- .../indexing/overlord/supervisor/SupervisorResourceTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index 2fb30ed33d81..1ea1f3fe53f3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -728,7 +728,7 @@ public void testResetSupervisorAndBackfill() streamSupervisor.submitBackfillTask( EasyMock.anyObject(), EasyMock.anyObject(), - null + EasyMock.isNull() ); replayAll(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 3e4d64838088..b21438a59348 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -1343,7 +1343,7 @@ public void testResetOffsetsAndBackfill() // Test 200 - Success EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(1); EasyMock.expect(supervisorManager.resetSupervisorAndBackfill( - EasyMock.capture(id1), null + EasyMock.capture(id1), EasyMock.isNull() )).andReturn(expectedResult); replayAll();