From dfda88c4aaa210a497b9059ff8240698df18289e Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Thu, 23 Apr 2026 16:29:59 -0700 Subject: [PATCH 01/52] Initial implementation for BoundedStreamConfig --- .../indexing/StreamIndexTestBase.java | 1 + .../kinesis/KinesisDataFormatsTest.java | 1 + .../supervisor/RabbitStreamSupervisor.java | 29 ++ .../RabbitStreamSupervisorIOConfig.java | 7 +- .../RabbitStreamSupervisorTest.java | 3 + .../kafka/supervisor/KafkaSupervisor.java | 25 ++ .../supervisor/KafkaSupervisorIOConfig.java | 7 +- .../supervisor/KafkaIOConfigBuilder.java | 1 + .../KafkaSupervisorIOConfigTest.java | 3 + .../kafka/supervisor/KafkaSupervisorTest.java | 8 +- .../kinesis/supervisor/KinesisSupervisor.java | 23 + .../supervisor/KinesisSupervisorIOConfig.java | 7 +- .../kinesis/KinesisSamplerSpecTest.java | 2 + .../supervisor/KinesisSupervisorTest.java | 8 + .../supervisor/BoundedStreamConfig.java | 90 ++++ .../supervisor/SeekableStreamSupervisor.java | 397 +++++++++++++++++- .../SeekableStreamSupervisorIOConfig.java | 17 +- .../SeekableStreamSupervisorSpec.java | 39 ++ .../SeekableStreamSamplerSpecTest.java | 1 + .../SeekableStreamSupervisorIOConfigTest.java | 13 +- .../SeekableStreamSupervisorSpecTest.java | 4 + .../SeekableStreamSupervisorStateTest.java | 46 +- .../SeekableStreamSupervisorTestBase.java | 19 + .../supervisor/SupervisorStateManager.java | 15 +- 24 files changed, 747 insertions(+), 19 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexTestBase.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexTestBase.java index 821d776947dc..ee826936b2b4 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexTestBase.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/StreamIndexTestBase.java @@ -120,6 +120,7 @@ protected KinesisSupervisorSpec createKinesisSupervisor(KinesisResource kinesis, Period.seconds(60), null, null, null, null, null, null, null, null, false, + null, null ), Map.of(), diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java index 2c5cfb67d9c3..9a37c73573a6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/kinesis/KinesisDataFormatsTest.java @@ -87,6 +87,7 @@ private KinesisSupervisorSpec createKinesisSupervisorSpec(String dataSource, Str Period.seconds(5), null, null, null, null, null, null, null, null, false, + null, null ), Map.of(), 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..f72e55147548 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 @@ -362,6 +362,35 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() return false; } + @Override + protected boolean isOffsetAtOrBeyond(Long current, Long target) + { + throw new UnsupportedOperationException( + "Bounded stream processing is not yet supported for RabbitMQ. " + + "This feature is currently only available for Kafka supervisors." + ); + } + + @Override + protected String createPartitionIdFromString(String partitionIdString) + { + // RabbitMQ uses String as partition ID, so just return the string as-is + return partitionIdString; + } + + @Override + protected Long createSequenceOffsetFromObject(Object offsetObj) + { + // RabbitMQ uses Long as sequence offset + if (offsetObj instanceof Number) { + return ((Number) offsetObj).longValue(); + } + if (offsetObj instanceof String) { + return Long.parseLong((String) offsetObj); + } + throw new IllegalArgumentException("Cannot convert " + offsetObj.getClass() + " to Long offset"); + } + @Override public LagStats computeLagStats() { 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..20c4e92988b6 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 @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.data.input.InputFormat; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; import org.apache.druid.indexing.seekablestream.supervisor.LagAggregator; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; @@ -66,7 +67,8 @@ public RabbitStreamSupervisorIOConfig( @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod, @JsonProperty("lateMessageRejectionStartDateTime") DateTime lateMessageRejectionStartDateTime, @JsonProperty("stopTaskCount") Integer stopTaskCount, - @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas + @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas, + @Nullable @JsonProperty("boundedStreamConfig") BoundedStreamConfig boundedStreamConfig ) { super( @@ -86,7 +88,8 @@ public RabbitStreamSupervisorIOConfig( lateMessageRejectionStartDateTime, new IdleConfig(null, null), stopTaskCount, - serverPriorityToReplicas + serverPriorityToReplicas, + boundedStreamConfig ); this.consumerProperties = consumerProperties; diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index 82e0b164471a..d66313848609 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -212,6 +212,7 @@ private RabbitStreamSupervisor getSupervisor( earlyMessageRejectionPeriod, // early message rejection null, // latemessagerejectionstartdatetime 1, + null, null ); RabbitStreamIndexTaskClientFactory clientFactory = new RabbitStreamIndexTaskClientFactory(null, @@ -278,6 +279,7 @@ public void testRecordSupplier() null, // early message rejection null, // latemessagerejectionstartdatetime 1, + null, null ); RabbitStreamIndexTaskClientFactory clientFactory = new RabbitStreamIndexTaskClientFactory(null, @@ -421,6 +423,7 @@ public void testCreateTaskIOConfig() null, // early message rejection null, // latemessagerejectionstartdatetime 1, + null, null ) ); 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 e2f62ed8d750..d85e56fc4918 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 @@ -390,6 +390,31 @@ protected boolean isShardExpirationMarker(Long seqNum) return false; } + @Override + protected boolean isOffsetAtOrBeyond(Long current, Long target) + { + return current >= target; + } + + @Override + protected KafkaTopicPartition createPartitionIdFromString(String partitionIdString) + { + return KafkaTopicPartition.fromString(partitionIdString); + } + + @Override + protected Long createSequenceOffsetFromObject(Object offsetObj) + { + // Jackson may deserialize numbers as Integer if they fit, but Kafka needs Long + if (offsetObj instanceof Number) { + return ((Number) offsetObj).longValue(); + } + if (offsetObj instanceof String) { + return Long.parseLong((String) offsetObj); + } + throw new IllegalArgumentException("Cannot convert " + offsetObj.getClass() + " to Long offset"); + } + @Override protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() { 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..7e9fffb7e79c 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 @@ -26,6 +26,7 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.seekablestream.extension.KafkaConfigOverrides; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; import org.apache.druid.indexing.seekablestream.supervisor.LagAggregator; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; @@ -78,7 +79,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("boundedStreamConfig") BoundedStreamConfig boundedStreamConfig ) { super( @@ -98,7 +100,8 @@ public KafkaSupervisorIOConfig( lateMessageRejectionStartDateTime, idleConfig, stopTaskCount, - serverPriorityToReplicas + serverPriorityToReplicas, + boundedStreamConfig ); 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/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 08491caa8ff5..262f4b39100d 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 @@ -381,6 +381,29 @@ protected boolean isShardExpirationMarker(String seqNum) return KinesisSequenceNumber.EXPIRED_MARKER.equals(seqNum); } + @Override + protected boolean isOffsetAtOrBeyond(String current, String target) + { + throw new UnsupportedOperationException( + "Bounded stream processing is not yet supported for Kinesis. " + + "This feature is currently only available for Kafka supervisors." + ); + } + + @Override + protected String createPartitionIdFromString(String partitionIdString) + { + // Kinesis uses String as partition ID, so just return the string as-is + return partitionIdString; + } + + @Override + protected String createSequenceOffsetFromObject(Object offsetObj) + { + // Kinesis uses String as sequence offset + return offsetObj.toString(); + } + @Override protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() { 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..e20d3a261cfb 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 @@ -26,6 +26,7 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig; import org.apache.druid.indexing.kinesis.KinesisRegion; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; import org.apache.druid.indexing.seekablestream.supervisor.LagAggregator; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; @@ -79,7 +80,8 @@ public KinesisSupervisorIOConfig( @JsonProperty("awsExternalId") String awsExternalId, @Nullable @JsonProperty("autoScalerConfig") AutoScalerConfig autoScalerConfig, @JsonProperty("deaggregate") @Deprecated boolean deaggregate, - @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas + @Nullable @JsonProperty("serverPriorityToReplicas") Map serverPriorityToReplicas, + @Nullable @JsonProperty("boundedStreamConfig") BoundedStreamConfig boundedStreamConfig ) { super( @@ -99,7 +101,8 @@ public KinesisSupervisorIOConfig( lateMessageRejectionStartDateTime, new IdleConfig(null, null), null, - serverPriorityToReplicas + serverPriorityToReplicas, + boundedStreamConfig ); this.endpoint = endpoint != null diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index 4c9f4a1c8e2c..6f4410f3a8be 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -140,6 +140,7 @@ public void testSample() throws InterruptedException null, null, false, + null, null ), null, @@ -195,6 +196,7 @@ public void testGetInputSourceResources() null, null, false, + null, null ), null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 522aebad4cf6..9c82671abccb 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -468,6 +468,7 @@ public void testRecordSupplier() null, null, false, + null, null ); KinesisIndexTaskClientFactory clientFactory = new KinesisIndexTaskClientFactory(null, OBJECT_MAPPER); @@ -534,6 +535,7 @@ public void testKinesisIOConfigInitAndAutoscalerConfigCreation() null, null, false, + null, null ); @@ -562,6 +564,7 @@ public void testKinesisIOConfigInitAndAutoscalerConfigCreation() null, OBJECT_MAPPER.convertValue(new HashMap<>(), AutoScalerConfig.class), false, + null, null ); @@ -4219,6 +4222,7 @@ public void testCorrectInputSources() null, null, false, + null, null ), null, @@ -5184,6 +5188,7 @@ private TestableKinesisSupervisor getTestableSupervisor( null, null, false, + null, null ); @@ -5327,6 +5332,7 @@ private TestableKinesisSupervisor getTestableSupervisor( null, autoScalerConfig, false, + null, null ); @@ -5414,6 +5420,7 @@ private TestableKinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( null, null, false, + null, null ); @@ -5503,6 +5510,7 @@ private KinesisSupervisor createSupervisor( null, null, false, + null, null ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java new file mode 100644 index 000000000000..6d8caf0686d3 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java @@ -0,0 +1,90 @@ +/* + * 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.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.Map; + +/** + * Configuration for bounded (one-time) stream processing with explicit start/end offsets. + * + * When configured, the supervisor will: + * 1. Create tasks starting at the specified startSequenceNumbers + * 2. Tasks will automatically stop when they reach endSequenceNumbers + * 3. Supervisor will not recreate tasks after they complete + * 4. Supervisor will auto-terminate when all tasks are done + * + * This is useful for: + * - Backfill processing + * - Historical reprocessing + * - One-time migration tasks + */ +public class BoundedStreamConfig +{ + private final Map startSequenceNumbers; // Partition -> Start Offset + private final Map endSequenceNumbers; // Partition -> End Offset + + @JsonCreator + public BoundedStreamConfig( + @JsonProperty("startSequenceNumbers") Map startSequenceNumbers, + @JsonProperty("endSequenceNumbers") Map endSequenceNumbers + ) + { + this.startSequenceNumbers = Preconditions.checkNotNull(startSequenceNumbers, "startSequenceNumbers"); + this.endSequenceNumbers = Preconditions.checkNotNull(endSequenceNumbers, "endSequenceNumbers"); + + // Validation + Preconditions.checkArgument( + !startSequenceNumbers.isEmpty(), + "startSequenceNumbers cannot be empty" + ); + + Preconditions.checkArgument( + startSequenceNumbers.keySet().equals(endSequenceNumbers.keySet()), + "startSequenceNumbers and endSequenceNumbers must have matching partition sets. Start: %s, End: %s", + startSequenceNumbers.keySet(), + endSequenceNumbers.keySet() + ); + } + + @JsonProperty + public Map getStartSequenceNumbers() + { + return startSequenceNumbers; + } + + @JsonProperty + public Map getEndSequenceNumbers() + { + return endSequenceNumbers; + } + + @Override + public String toString() + { + return "BoundedStreamConfig{" + + "startSequenceNumbers=" + startSequenceNumbers + + ", endSequenceNumbers=" + endSequenceNumbers + + '}'; + } +} 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 58d366d2306e..b006e90119d1 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 @@ -206,6 +206,10 @@ public class TaskGroup // task groups have nothing but closed partitions in their assignments. final ImmutableMap unfilteredStartingSequencesForSequenceName; + // End sequences for bounded mode - null for streaming mode + @Nullable + final ImmutableMap endSequences; + final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); final ConcurrentHashMap taskIdToServerPriority = new ConcurrentHashMap<>(); final DateTime minimumMessageTime; @@ -230,6 +234,37 @@ public class TaskGroup groupId, startingSequences, unfilteredStartingSequencesForSequenceName, + null, // endSequences - null for streaming mode + minimumMessageTime, + maximumMessageTime, + exclusiveStartSequenceNumberPartitions, + generateSequenceName( + unfilteredStartingSequencesForSequenceName == null + ? startingSequences + : unfilteredStartingSequencesForSequenceName, + minimumMessageTime, + maximumMessageTime, + spec.getDataSchema(), + taskTuningConfig + ) + ); + } + + TaskGroup( + int groupId, + ImmutableMap startingSequences, + @Nullable ImmutableMap unfilteredStartingSequencesForSequenceName, + @Nullable ImmutableMap endSequences, + @Nullable DateTime minimumMessageTime, + @Nullable DateTime maximumMessageTime, + @Nullable Set exclusiveStartSequenceNumberPartitions + ) + { + this( + groupId, + startingSequences, + unfilteredStartingSequencesForSequenceName, + endSequences, minimumMessageTime, maximumMessageTime, exclusiveStartSequenceNumberPartitions, @@ -249,6 +284,7 @@ public class TaskGroup int groupId, ImmutableMap startingSequences, @Nullable ImmutableMap unfilteredStartingSequencesForSequenceName, + @Nullable ImmutableMap endSequences, DateTime minimumMessageTime, DateTime maximumMessageTime, Set exclusiveStartSequenceNumberPartitions, @@ -260,6 +296,7 @@ public class TaskGroup this.unfilteredStartingSequencesForSequenceName = unfilteredStartingSequencesForSequenceName == null ? startingSequences : unfilteredStartingSequencesForSequenceName; + this.endSequences = endSequences; this.minimumMessageTime = minimumMessageTime; this.maximumMessageTime = maximumMessageTime; this.checkpointSequences.put(0, startingSequences); @@ -1347,6 +1384,17 @@ public void tryInit() try { recordSupplier = setupRecordSupplier(); + // Initialize bounded partitions BEFORE first run + if (ioConfig.isBounded()) { + try { + initializeBoundedPartitionGroups(); + } + catch (Exception e) { + log.error(e, "Failed to initialize bounded partition groups"); + throw new RuntimeException(e); + } + } + exec.submit( () -> { try { @@ -1851,6 +1899,12 @@ public void runInternal() } } + // Check for bounded completion after tasks have been created/managed + if (isBoundedWorkComplete()) { + handleBoundedCompletion(); + return; + } + logDebugReport(); } catch (Exception e) { @@ -2338,6 +2392,17 @@ public Boolean apply(Pair endSequences = null; + if (seekableStreamIndexTask.getIOConfig().getEndSequenceNumbers() != null) { + endSequences = ImmutableMap.copyOf( + seekableStreamIndexTask.getIOConfig() + .getEndSequenceNumbers() + .getPartitionSequenceNumberMap() + ); + } + return new TaskGroup( taskGroupId, ImmutableMap.copyOf( @@ -2346,6 +2411,7 @@ public Boolean apply(Pair previousPartitionIds = new ArrayList<>(partitionIds); Set partitionIdsFromSupplier; recordSupplierLock.lock(); @@ -3161,6 +3234,59 @@ protected Map getLatestSequencesFromStream( return new HashMap<>(); } + /** + * Converts a map with string keys from BoundedStreamConfig to a properly typed map. + * The BoundedStreamConfig uses Map which Jackson deserializes as Map. + * This method converts string keys to partition IDs and offset values to the appropriate types. + * + * @param rawMap the raw map from BoundedStreamConfig + * @return a map with properly typed partition IDs and sequence offsets + */ + private Map convertBoundedConfigMap(Map rawMap) + { + Map result = new HashMap<>(); + for (Map.Entry entry : rawMap.entrySet()) { + PartitionIdType partition = createPartitionIdFromString(entry.getKey().toString()); + SequenceOffsetType offset = createSequenceOffsetFromObject(entry.getValue()); + result.put(partition, offset); + } + return result; + } + + /** + * Initialize partitionGroups from bounded config instead of from stream discovery. + * This prevents the supervisor from trying to recreate tasks as they complete. + * Only called when in bounded mode during supervisor startup. + */ + private void initializeBoundedPartitionGroups() + { + if (!ioConfig.isBounded()) { + return; + } + + BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); + Map configuredPartitions = + convertBoundedConfigMap(boundedConfig.getStartSequenceNumbers()); + + for (PartitionIdType partition : configuredPartitions.keySet()) { + int taskGroupId = getTaskGroupIdForPartition(partition); + + partitionGroups.computeIfAbsent(taskGroupId, k -> new HashSet<>()).add(partition); + partitionIds.add(partition); + partitionOffsets.put(partition, getNotSetMarker()); + + log.info("Bounded mode: initialized partition[%s] in taskGroup[%d]", partition, taskGroupId); + } + + assignRecordSupplierToPartitionIds(); + + log.info( + "Bounded mode: initialized [%d] partitions in [%d] task groups", + configuredPartitions.size(), + partitionGroups.size() + ); + } + private void assignRecordSupplierToPartitionIds() { recordSupplierLock.lock(); @@ -4052,6 +4178,29 @@ private void createNewTasks() throws JsonProcessingException // check that there is a current task group for each group of partitions in [partitionGroups] for (Integer groupId : partitionGroups.keySet()) { if (!activelyReadingTaskGroups.containsKey(groupId)) { + + // In bounded mode, distinguish between completion and failure + if (ioConfig.isBounded()) { + if (hasTaskGroupReachedBoundedEnd(groupId)) { + // Task group completed successfully - don't recreate + log.debug( + "Bounded taskGroup[%d] has reached end offsets, skipping recreation", + groupId + ); + continue; // Skip creating new task group + } else { + // Task group hasn't reached end - task must have failed, recreate it + log.info( + "Bounded taskGroup[%d] has not reached end offsets (current: %s, target: %s). " + + "Task may have failed, recreating to continue processing.", + groupId, + getCurrentOffsetsForGroup(groupId), + getEndOffsetsForGroup(groupId) + ); + // Fall through to create new task group + } + } + log.info("Creating new taskGroup[%d] for partitions[%s].", groupId, partitionGroups.get(groupId)); final DateTime minimumMessageTime; if (ioConfig.getLateMessageRejectionStartDateTime().isPresent()) { @@ -4118,13 +4267,25 @@ private void createNewTasks() throws JsonProcessingException .collect(Collectors.toSet()); } - log.info("Initializing taskGroup[%d] with startingOffsets[%s].", groupId, simpleStartingOffsets); + // NEW: Extract end offsets for bounded mode + ImmutableMap endOffsets = null; + if (ioConfig.isBounded()) { + endOffsets = ImmutableMap.copyOf(getEndOffsetsForGroup(groupId)); + } + + log.info( + "Initializing taskGroup[%d] with startingOffsets[%s] and endOffsets[%s]", + groupId, + simpleStartingOffsets, + endOffsets + ); activelyReadingTaskGroups.put( groupId, new TaskGroup( groupId, simpleStartingOffsets, simpleUnfilteredStartingOffsets, + endOffsets, minimumMessageTime, maximumMessageTime, exclusiveStartSequenceNumberPartitions @@ -4188,6 +4349,8 @@ private Map> generate int groupId ) { + // Existing logic for both streaming and bounded mode + // Bounded mode will fall back to bounded start offsets in getOffsetFromStorageForPartition() ImmutableMap.Builder> builder = ImmutableMap.builder(); final Map metadataOffsets = getOffsetsFromMetadataStorage(); for (PartitionIdType partitionId : partitionGroups.get(groupId)) { @@ -4255,6 +4418,23 @@ private OrderedSequenceNumber getOffsetFromStorageForPartiti } return makeSequenceNumber(sequence, useExclusiveStartSequenceNumberForNonFirstSequence()); } else { + // NEW: In bounded mode, if no checkpoint exists (task failed before first checkpoint), + // fall back to bounded start offset + if (ioConfig.isBounded()) { + BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); + Map startOffsets = + convertBoundedConfigMap(boundedConfig.getStartSequenceNumbers()); + SequenceOffsetType startOffset = startOffsets.get(partition); + if (startOffset != null) { + log.info( + "Bounded mode: no checkpoint found for partition[%s], using configured start offset[%s]", + partition, + startOffset + ); + return makeSequenceNumber(startOffset, false); + } + } + boolean useEarliestSequenceNumber = ioConfig.isUseEarliestSequenceNumber(); if (subsequentlyDiscoveredPartitions.contains(partition)) { log.info( @@ -4298,6 +4478,179 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { return Collections.emptyMap(); } + /** + * Check if all partitions in a task group have reached their bounded end offsets. + * Used to determine if the task group completed successfully vs failed midway. + * + * @param groupId The task group ID to check + * @return true if all partitions in the group have reached their end offsets, false otherwise + */ + private boolean hasTaskGroupReachedBoundedEnd(int groupId) + { + BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); + Map endOffsets = + convertBoundedConfigMap(boundedConfig.getEndSequenceNumbers()); + Map currentOffsets = getOffsetsFromMetadataStorage(); + + log.info( + "Bounded mode: checking completion for taskGroup[%d]. Current offsets from metadata: %s, End offsets: %s", + groupId, + currentOffsets, + endOffsets + ); + + if (currentOffsets == null || currentOffsets.isEmpty()) { + log.debug("No checkpointed offsets found, taskGroup[%d] has not completed", groupId); + return false; // No progress yet, task hasn't completed + } + + Set partitionsInGroup = partitionGroups.get(groupId); + if (partitionsInGroup == null || partitionsInGroup.isEmpty()) { + return false; + } + + // Check if ALL partitions in this group have reached their end offsets + for (PartitionIdType partition : partitionsInGroup) { + SequenceOffsetType endOffset = endOffsets.get(partition); + SequenceOffsetType currentOffset = currentOffsets.get(partition); + + if (currentOffset == null) { + log.debug( + "Partition[%s] in taskGroup[%d] has no checkpointed offset, not complete", + partition, + groupId + ); + return false; // Partition hasn't started processing + } + + if (!isOffsetAtOrBeyond(currentOffset, endOffset)) { + log.debug( + "Partition[%s] in taskGroup[%d] at offset[%s], has not reached end[%s]", + partition, + groupId, + currentOffset, + endOffset + ); + return false; // This partition hasn't reached its end + } + } + + log.info( + "All partitions in taskGroup[%d] have reached their end offsets", + groupId + ); + return true; // All partitions have reached their end offsets + } + + /** + * Get current offsets for all partitions in a task group from metadata storage. + */ + private Map getCurrentOffsetsForGroup(int groupId) + { + Map allOffsets = getOffsetsFromMetadataStorage(); + if (allOffsets == null || allOffsets.isEmpty()) { + return Collections.emptyMap(); + } + + Set partitionsInGroup = partitionGroups.get(groupId); + if (partitionsInGroup == null) { + return Collections.emptyMap(); + } + + return partitionsInGroup.stream() + .filter(allOffsets::containsKey) + .collect(Collectors.toMap( + p -> p, + allOffsets::get + )); + } + + /** + * Get end offsets for all partitions in a task group from bounded config. + */ + private Map getEndOffsetsForGroup(int groupId) + { + BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); + Map endOffsets = + convertBoundedConfigMap(boundedConfig.getEndSequenceNumbers()); + Set partitionsInGroup = partitionGroups.get(groupId); + + if (partitionsInGroup == null) { + return Collections.emptyMap(); + } + + return partitionsInGroup.stream() + .filter(endOffsets::containsKey) + .collect(Collectors.toMap( + p -> p, + endOffsets::get + )); + } + + /** + * Check if all bounded tasks have completed. + * Called after createNewTasks() in runInternal to ensure tasks have been created first. + * + * For bounded supervisors, we determine completion by checking if new tasks would be created. + * In createNewTasks(), bounded mode checks hasTaskGroupReachedBoundedEnd() before creating tasks. + * If that returns true (offsets reached), no new tasks are created. + * + * So completion is: no active tasks, no pending tasks, and createNewTasks() chose not to create any. + * This is indicated by empty task groups after createNewTasks() has run. + * + * We do NOT separately check metadata storage here because: + * 1. Metadata may contain stale offsets from previous supervisor runs + * 2. createNewTasks() already does the offset checking logic + * 3. If tasks were killed/failed and work is incomplete, createNewTasks() will recreate them + * + * @return true if all bounded work is complete, false otherwise + */ + private boolean isBoundedWorkComplete() + { + if (!ioConfig.isBounded()) { + return false; + } + + // Check if task groups are empty (no tasks active or pending) + boolean noActiveTasks = activelyReadingTaskGroups.isEmpty(); + boolean noPendingTasks = pendingCompletionTaskGroups.values().stream().allMatch(List::isEmpty); + + if (!noActiveTasks || !noPendingTasks) { + return false; + } + + // At this point, no tasks are running. Since createNewTasks() already ran, + // if tasks aren't running it means either: + // A) Tasks completed successfully and offset targets were reached (don't recreate) + // B) Tasks failed/killed and haven't reached targets (will recreate next run) + // + // To distinguish, we check if createNewTasks() would create new tasks. + // If hasTaskGroupReachedBoundedEnd() returns false for any group, createNewTasks() + // will create tasks next iteration, so we're not complete. + for (Integer groupId : partitionGroups.keySet()) { + if (!hasTaskGroupReachedBoundedEnd(groupId)) { + log.debug("TaskGroup[%d] has not reached bounded end, tasks will be recreated", groupId); + return false; + } + } + + // All groups have reached their end offsets and no tasks are running. + // Work is complete! + log.info("All bounded tasks completed for supervisor[%s]", supervisorId); + return true; + } + + /** + * Handle bounded processing completion by shutting down the supervisor. + * At this point, all task groups are already empty (verified by isBoundedWorkComplete), + * so we just need to mark the supervisor as completed. + */ + private void handleBoundedCompletion() + { + log.info("Bounded processing complete for supervisor[%s]. Marking as COMPLETED.", supervisorId); + stateManager.maybeSetState(SupervisorStateManager.BasicState.COMPLETED); + } + protected DataSourceMetadata retrieveDataSourceMetadata() { return indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(supervisorId); @@ -4333,9 +4686,18 @@ private void createTasksForGroup(int groupId, int replicas) TaskGroup group = activelyReadingTaskGroups.get(groupId); Map startPartitions = group.startingSequences; Map endPartitions = new HashMap<>(); - for (PartitionIdType partition : startPartitions.keySet()) { - endPartitions.put(partition, getEndOfPartitionMarker()); + + if (group.endSequences != null && !group.endSequences.isEmpty()) { + // Bounded mode: use explicit end offsets from task group + endPartitions.putAll(group.endSequences); + log.info("Creating bounded tasks for taskGroup[%d] with endOffsets: %s", groupId, group.endSequences); + } else { + // Streaming mode: use exclusive end (effectively no end) + for (PartitionIdType partition : startPartitions.keySet()) { + endPartitions.put(partition, getEndOfPartitionMarker()); + } } + Set exclusiveStartSequenceNumberPartitions = activelyReadingTaskGroups .get(groupId) .exclusiveStartSequenceNumberPartitions; @@ -4692,6 +5054,25 @@ protected abstract List serverPrioritiesToAssign ) throws JsonProcessingException; + /** + * Converts a string representation of a partition ID to the typed partition ID. + * Used for deserializing bounded stream config where partition keys come as strings. + * + * @param partitionIdString string representation of partition ID + * @return typed partition ID + */ + protected abstract PartitionIdType createPartitionIdFromString(String partitionIdString); + + /** + * Converts an object (typically Number) to the typed sequence offset. + * Used for deserializing bounded stream config where offset values may come as Integer, Long, String, etc. + * Jackson may deserialize numeric values as Integer if they fit, but implementations like Kafka need Long. + * + * @param offsetObj the offset object from deserialization + * @return typed sequence offset + */ + protected abstract SequenceOffsetType createSequenceOffsetFromObject(Object offsetObj); + /** * calculates the taskgroup id that the given partition belongs to. * different between Kafka/Kinesis since Kinesis uses String as partition id @@ -5049,6 +5430,16 @@ protected LagStats aggregatePartitionLags(Map partitionLa */ protected abstract boolean isShardExpirationMarker(SequenceOffsetType seqNum); + /** + * Compares if current offset has reached or exceeded the target offset. + * Used to determine if a bounded task group has completed successfully. + * + * @param current Current offset from metadata storage + * @param target Target end offset from bounded config + * @return true if current >= target + */ + protected abstract boolean isOffsetAtOrBeyond(SequenceOffsetType current, SequenceOffsetType target); + /** * Returns true if the start sequence number should be exclusive for the non-first sequences for the whole partition. * For example, in Kinesis, the start offsets are inclusive for the first sequence, but exclusive for following 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 421a885b294d..4bd806c49595 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 BoundedStreamConfig boundedStreamConfig; 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 BoundedStreamConfig boundedStreamConfig ) { this.stream = Preconditions.checkNotNull(stream, "stream cannot be null"); @@ -128,6 +130,7 @@ public SeekableStreamSupervisorIOConfig( this.idleConfig = idleConfig; this.serverPriorityToReplicas = serverPriorityToReplicas; + this.boundedStreamConfig = boundedStreamConfig; if (this.serverPriorityToReplicas != null) { int serverPriorityReplicas = 0; @@ -283,4 +286,16 @@ public int getMaxAllowedStops() } return stopTaskCount == null ? taskCount : stopTaskCount; } + + @Nullable + @JsonProperty + public BoundedStreamConfig getBoundedStreamConfig() + { + return boundedStreamConfig; + } + + public boolean isBounded() + { + return boundedStreamConfig != null; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index f21e073f6c4c..bee2994d7b6c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -37,6 +37,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.NoopTaskAutoScaler; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -49,6 +50,8 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec { + private static final Logger log = new Logger(SeekableStreamSupervisorSpec.class); + protected static final String ILLEGAL_INPUT_SOURCE_UPDATE_ERROR_MESSAGE = "Update of the input source stream from [%s] to [%s] is not supported for a running supervisor." + "%nTo perform the update safely, follow these steps:" @@ -259,6 +262,42 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept if (!this.getSource().equals(other.getSource())) { throw InvalidInput.exception(ILLEGAL_INPUT_SOURCE_UPDATE_ERROR_MESSAGE, this.getSource(), other.getSource()); } + + // Validate bounded stream configuration + validateBoundedStreamConfig(other); + } + + /** + * Validates bounded stream configuration for the supervisor spec. + * + * @param spec the supervisor spec to validate + * @throws DruidException if the bounded stream configuration is invalid + */ + protected void validateBoundedStreamConfig(SeekableStreamSupervisorSpec spec) throws DruidException + { + SeekableStreamSupervisorIOConfig ioConfig = spec.getIoConfig(); + + if (ioConfig.isBounded()) { + // Validate partition consistency + BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); + if (!boundedConfig.getStartSequenceNumbers().keySet().equals(boundedConfig.getEndSequenceNumbers().keySet())) { + throw InvalidInput.exception( + "Bounded stream config has mismatched partitions. Start: %s, End: %s", + boundedConfig.getStartSequenceNumbers().keySet(), + boundedConfig.getEndSequenceNumbers().keySet() + ); + } + + // Warn if useConcurrentLocks is not enabled + Map context = spec.getContext(); + if (context == null || !Boolean.TRUE.equals(context.get("useConcurrentLocks"))) { + log.warn( + "Bounded stream processing without 'useConcurrentLocks=true' may fail " + + "if other supervisors are running or segments already exist for these intervals. " + + "Consider setting useConcurrentLocks=true in the supervisor context." + ); + } + } } @Override 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 5af21ca7b6aa..6fd6307e50c7 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,6 +333,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 7c20855b033d..0efad16bc276 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 @@ -785,6 +785,7 @@ public void testSeekableStreamSupervisorSpecWithScaleDisable() throws Interrupte null, null, null, + null, null ) { @@ -843,6 +844,7 @@ public void testEnablingIdleBeviourPerSupervisorWithOverlordConfigEnabled() null, new IdleConfig(true, null), null, + null, null ) { @@ -1455,6 +1457,7 @@ private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scal null, null, null, + null, null ) { @@ -1477,6 +1480,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 d59e2711ce88..a17d59759e76 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,6 +697,7 @@ public void testIdleStateTransition() throws Exception null, new IdleConfig(true, 200L), null, + null, null ) { @@ -805,6 +806,7 @@ public void testIdleOnStartUpAndTurnsToRunningAfterLagUpdates() null, new IdleConfig(true, 200L), null, + null, null ) { @@ -1105,6 +1107,7 @@ public void testCheckpointForActiveTaskGroup() throws InterruptedException, Json null, new IdleConfig(true, 200L), null, + null, null ) {}; @@ -1324,6 +1327,7 @@ public void testEarlyStoppingOfTaskGroupBasedOnStopTaskCount() throws Interrupte null, new IdleConfig(true, 200L), stopTaskCount, + null, null ) { @@ -1560,6 +1564,7 @@ public void testSupervisorStopTaskGroupEarly() throws JsonProcessingException, I null, new IdleConfig(true, 200L), null, + null, null ) { @@ -2609,6 +2614,24 @@ public LagStats computeLagStats() { return new LagStats(0, 0, 0); } + + @Override + protected boolean isOffsetAtOrBeyond(String current, String target) + { + return Long.parseLong(current) >= Long.parseLong(target); + } + + @Override + protected String createPartitionIdFromString(String partitionIdString) + { + return partitionIdString; + } + + @Override + protected String createSequenceOffsetFromObject(Object offsetObj) + { + return offsetObj.toString(); + } }; supervisor.scheduleReporting(executorService); EasyMock.verify(executorService, spec); @@ -2715,6 +2738,7 @@ private void expectEmitterSupervisor(boolean suspended) null, null, null, + null, null ) { @@ -2779,6 +2803,7 @@ public void testMaxAllowedStopsWithStopTaskCountRatio() null, null, 1, // ensure this is overridden + null, null ) { @@ -2934,7 +2959,8 @@ private static SeekableStreamSupervisorIOConfig createSupervisorIOConfig( null, null, null, - serverPriorityToReplicas + serverPriorityToReplicas, + null ) { }; @@ -3273,6 +3299,24 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() { return false; } + + @Override + protected boolean isOffsetAtOrBeyond(String current, String target) + { + return Long.parseLong(current) >= Long.parseLong(target); + } + + @Override + protected String createPartitionIdFromString(String partitionIdString) + { + return partitionIdString; + } + + @Override + protected String createSequenceOffsetFromObject(Object offsetObj) + { + return offsetObj.toString(); + } } private class TestSeekableStreamSupervisor extends BaseTestSeekableStreamSupervisor 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..dad426a755c5 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 @@ -328,6 +328,24 @@ public int getPartitionCount() { return partitionNumbers; } + + @Override + protected boolean isOffsetAtOrBeyond(String current, String target) + { + return Long.parseLong(current) >= Long.parseLong(target); + } + + @Override + protected String createPartitionIdFromString(String partitionIdString) + { + return partitionIdString; + } + + @Override + protected String createSequenceOffsetFromObject(Object offsetObj) + { + return offsetObj.toString(); + } } class StateOverrideTestSeekableStreamSupervisor extends TestSeekableStreamSupervisor @@ -537,6 +555,7 @@ protected SeekableStreamSupervisorIOConfig createIOConfig(int taskCount, CostBas null, null, null, + null, null ) { diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java index f64f390fc566..437ea6b92922 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java @@ -64,7 +64,8 @@ public enum BasicState implements State RUNNING(true, false), IDLE(true, false), SUSPENDED(true, false), - STOPPING(true, false); + STOPPING(true, false), + COMPLETED(true, false); private final boolean healthy; private final boolean firstRunOnly; @@ -122,14 +123,18 @@ public SupervisorStateManager(SupervisorStateManagerConfig supervisorStateManage /** * Certain states are only valid if the supervisor hasn't had a successful iteration. This method checks if there's * been at least one successful iteration, and if applicable, sets supervisor state to an appropriate new state. - * A STOPPING supervisor cannot transition to any other state as this state is final. + * STOPPING and COMPLETED are terminal states that cannot transition to any other state. * This method must be thread-safe as multiple threads trying to update may lead to an invalid state. */ public synchronized void maybeSetState(State proposedState) { - if (BasicState.STOPPING.equals(this.supervisorState) || BasicState.STOPPING.equals(proposedState)) { - // STOPPING takes precedence over all other states - supervisorState = BasicState.STOPPING; + // Terminal states (STOPPING, COMPLETED) take precedence over all other states + if (BasicState.STOPPING.equals(this.supervisorState) || BasicState.COMPLETED.equals(this.supervisorState)) { + // Already in a terminal state, cannot transition + return; + } else if (BasicState.STOPPING.equals(proposedState) || BasicState.COMPLETED.equals(proposedState)) { + // Transitioning to a terminal state + supervisorState = proposedState; return; } From 4180acee9a8170c54159a41ad919856b737cb303 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Thu, 23 Apr 2026 16:34:35 -0700 Subject: [PATCH 02/52] Implement isOffsetAtOrBeyond for Rabbit and Kinesis --- .../rabbitstream/supervisor/RabbitStreamSupervisor.java | 6 ++---- .../indexing/kinesis/supervisor/KinesisSupervisor.java | 7 +++---- 2 files changed, 5 insertions(+), 8 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 f72e55147548..d3907f16008d 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 @@ -365,10 +365,8 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() @Override protected boolean isOffsetAtOrBeyond(Long current, Long target) { - throw new UnsupportedOperationException( - "Bounded stream processing is not yet supported for RabbitMQ. " + - "This feature is currently only available for Kafka supervisors." - ); + // RabbitMQ uses Long sequence numbers (delivery tags) + return current >= target; } @Override 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 262f4b39100d..d62872e01d51 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 @@ -384,10 +384,9 @@ protected boolean isShardExpirationMarker(String seqNum) @Override protected boolean isOffsetAtOrBeyond(String current, String target) { - throw new UnsupportedOperationException( - "Bounded stream processing is not yet supported for Kinesis. " + - "This feature is currently only available for Kafka supervisors." - ); + // Kinesis sequence numbers are comparable strings + // They can be compared lexicographically to determine order + return current.compareTo(target) >= 0; } @Override From 8cb75f699ea5228069aa49ef57443bbd693a247d Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 24 Apr 2026 10:21:12 -0700 Subject: [PATCH 03/52] Unit test coverage --- ...RabbitStreamSupervisorBoundedModeTest.java | 204 +++++++++++++++++ .../RabbitStreamSupervisorIOConfigTest.java | 76 +++++++ .../RabbitStreamSupervisorTest.java | 41 ++++ .../KafkaSupervisorBoundedModeTest.java | 206 ++++++++++++++++++ .../KafkaSupervisorIOConfigTest.java | 128 +++++++++++ .../kafka/supervisor/KafkaSupervisorTest.java | 92 ++++++++ .../KinesisSupervisorBoundedModeTest.java | 202 +++++++++++++++++ .../KinesisSupervisorIOConfigTest.java | 73 +++++++ .../supervisor/KinesisSupervisorTest.java | 95 ++++++++ .../supervisor/BoundedStreamConfigTest.java | 175 +++++++++++++++ .../SeekableStreamSupervisorIOConfigTest.java | 101 +++++++++ .../SupervisorStateManagerTest.java | 84 +++++++ 12 files changed, 1477 insertions(+) create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java create mode 100644 extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java new file mode 100644 index 000000000000..81420e7fe91f --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java @@ -0,0 +1,204 @@ +/* + * 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.rabbitstream.supervisor; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class RabbitStreamSupervisorBoundedModeTest +{ + @Test + public void testCreatePartitionIdFromString() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + String partition = supervisor.createPartitionIdFromString("queue-0"); + + Assert.assertEquals("queue-0", partition); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithInteger() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Long offset = supervisor.createSequenceOffsetFromObject(100); + + Assert.assertEquals(Long.valueOf(100L), offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithLong() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Long offset = supervisor.createSequenceOffsetFromObject(100L); + + Assert.assertEquals(Long.valueOf(100L), offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithString() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Long offset = supervisor.createSequenceOffsetFromObject("100"); + + Assert.assertEquals(Long.valueOf(100L), offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithInvalidType() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + IllegalArgumentException ex = Assert.assertThrows( + IllegalArgumentException.class, + () -> supervisor.createSequenceOffsetFromObject(new Object()) + ); + + Assert.assertTrue(ex.getMessage().contains("Cannot convert")); + } + + @Test + public void testIsOffsetAtOrBeyondEqual() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(100L, 100L)); + } + + @Test + public void testIsOffsetAtOrBeyondGreater() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(200L, 100L)); + } + + @Test + public void testIsOffsetAtOrBeyondLess() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); + } + + @Test + public void testConvertBoundedConfigMapWithIntegerValues() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("queue-0", 100); + rawMap.put("queue-1", 200); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(2, converted.size()); + Assert.assertEquals(Long.valueOf(100L), converted.get("queue-0")); + Assert.assertEquals(Long.valueOf(200L), converted.get("queue-1")); + } + + @Test + public void testConvertBoundedConfigMapWithStringValues() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("queue-0", "100"); + rawMap.put("queue-1", "200"); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(2, converted.size()); + Assert.assertEquals(Long.valueOf(100L), converted.get("queue-0")); + Assert.assertEquals(Long.valueOf(200L), converted.get("queue-1")); + } + + @Test + public void testConvertBoundedConfigMapWithMixedValues() + { + TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("queue-0", 100); + rawMap.put("queue-1", "200"); + rawMap.put("queue-2", 300L); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(3, converted.size()); + Assert.assertEquals(Long.valueOf(100L), converted.get("queue-0")); + Assert.assertEquals(Long.valueOf(200L), converted.get("queue-1")); + Assert.assertEquals(Long.valueOf(300L), converted.get("queue-2")); + } + + /** + * Minimal testable subclass that exposes protected methods for testing. + */ + private static class TestableRabbitStreamSupervisor extends RabbitStreamSupervisor + { + public TestableRabbitStreamSupervisor() + { + super( + null, + null, + null, + null, + null, + null, + null + ); + } + + @Override + public String createPartitionIdFromString(String partitionIdString) + { + return super.createPartitionIdFromString(partitionIdString); + } + + @Override + public Long createSequenceOffsetFromObject(Object offsetObj) + { + return super.createSequenceOffsetFromObject(offsetObj); + } + + @Override + public boolean isOffsetAtOrBeyond(Long current, Long target) + { + return super.isOffsetAtOrBeyond(current, target); + } + + public Map convertBoundedConfigMap(Map rawMap) + { + Map result = new HashMap<>(); + for (Map.Entry entry : rawMap.entrySet()) { + String partition = createPartitionIdFromString(entry.getKey().toString()); + Long offset = createSequenceOffsetFromObject(entry.getValue()); + result.put(partition, offset); + } + return result; + } + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java index 347152a4bb28..0dca7f084f0a 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java @@ -136,4 +136,80 @@ public void testURIRequired() throws Exception mapper.readValue(jsonStr, RabbitStreamSupervisorIOConfig.class); } + @Test + public void testBoundedModeSerdeWithIntegerOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"uri\": \"rabbitmq-stream://localhost:5552\",\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"queue-0\": 100, \"queue-1\": 200},\n" + + " \"endSequenceNumbers\": {\"queue-0\": 500, \"queue-1\": 600}\n" + + " }\n" + + "}"; + + RabbitStreamSupervisorIOConfig config = mapper.readValue(jsonStr, RabbitStreamSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } + + @Test + public void testBoundedModeSerdeWithStringOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"uri\": \"rabbitmq-stream://localhost:5552\",\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"queue-0\": \"100\", \"queue-1\": \"200\"},\n" + + " \"endSequenceNumbers\": {\"queue-0\": \"500\", \"queue-1\": \"600\"}\n" + + " }\n" + + "}"; + + RabbitStreamSupervisorIOConfig config = mapper.readValue(jsonStr, RabbitStreamSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } + + @Test + public void testBoundedModeSerdeWithMixedOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"uri\": \"rabbitmq-stream://localhost:5552\",\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"queue-0\": 100, \"queue-1\": \"200\"},\n" + + " \"endSequenceNumbers\": {\"queue-0\": 500, \"queue-1\": \"600\"}\n" + + " }\n" + + "}"; + + RabbitStreamSupervisorIOConfig config = mapper.readValue(jsonStr, RabbitStreamSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + } + + @Test + public void testUnboundedModeByDefault() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"uri\": \"rabbitmq-stream://localhost:5552\"\n" + + "}"; + + RabbitStreamSupervisorIOConfig config = mapper.readValue(jsonStr, RabbitStreamSupervisorIOConfig.class); + + Assert.assertFalse(config.isBounded()); + Assert.assertNull(config.getBoundedStreamConfig()); + } + } diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index d66313848609..b9ab01e13248 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -42,6 +42,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -464,4 +465,44 @@ public void test_doesTaskMatchSupervisor() Assert.assertFalse(supervisor.doesTaskMatchSupervisor(differentTaskType)); } + + @Test + public void testBoundedModeConfiguration() + { + ImmutableMap startOffsets = ImmutableMap.of( + "queue-0", 100, + "queue-1", 200 + ); + ImmutableMap endOffsets = ImmutableMap.of( + "queue-0", 500, + "queue-1", 600 + ); + + final RabbitStreamSupervisorIOConfig rabbitSupervisorIOConfig = new RabbitStreamSupervisorIOConfig( + STREAM, + URI, + INPUT_FORMAT, + 1, + 1, + new Period("PT1H"), + null, + null, + null, + new Period("PT30M"), + null, + null, + null, + null, + null, + null, + 1000, + null, + new BoundedStreamConfig(startOffsets, endOffsets) + ); + + Assert.assertTrue(rabbitSupervisorIOConfig.isBounded()); + Assert.assertNotNull(rabbitSupervisorIOConfig.getBoundedStreamConfig()); + Assert.assertEquals(2, rabbitSupervisorIOConfig.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, rabbitSupervisorIOConfig.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java new file mode 100644 index 000000000000..b654a68dadd6 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java @@ -0,0 +1,206 @@ +/* + * 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.kafka.supervisor; + +import org.apache.druid.data.input.kafka.KafkaTopicPartition; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class KafkaSupervisorBoundedModeTest +{ + @Test + public void testCreatePartitionIdFromString() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + KafkaTopicPartition partition = supervisor.createPartitionIdFromString("my-topic:5"); + + Assert.assertEquals("my-topic", partition.topic()); + Assert.assertEquals(5, partition.partition()); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithInteger() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Long offset = supervisor.createSequenceOffsetFromObject(100); + + Assert.assertEquals(Long.valueOf(100L), offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithLong() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Long offset = supervisor.createSequenceOffsetFromObject(100L); + + Assert.assertEquals(Long.valueOf(100L), offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithString() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Long offset = supervisor.createSequenceOffsetFromObject("100"); + + Assert.assertEquals(Long.valueOf(100L), offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithInvalidType() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + IllegalArgumentException ex = Assert.assertThrows( + IllegalArgumentException.class, + () -> supervisor.createSequenceOffsetFromObject(new Object()) + ); + + Assert.assertTrue(ex.getMessage().contains("Cannot convert")); + } + + @Test + public void testIsOffsetAtOrBeyondEqual() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(100L, 100L)); + } + + @Test + public void testIsOffsetAtOrBeyondGreater() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(200L, 100L)); + } + + @Test + public void testIsOffsetAtOrBeyondLess() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); + } + + @Test + public void testConvertBoundedConfigMapWithIntegerValues() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("my-topic:0", 100); + rawMap.put("my-topic:1", 200); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(2, converted.size()); + Assert.assertEquals(Long.valueOf(100L), converted.get(new KafkaTopicPartition(false, "my-topic", 0))); + Assert.assertEquals(Long.valueOf(200L), converted.get(new KafkaTopicPartition(false, "my-topic", 1))); + } + + @Test + public void testConvertBoundedConfigMapWithStringValues() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("my-topic:0", "100"); + rawMap.put("my-topic:1", "200"); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(2, converted.size()); + Assert.assertEquals(Long.valueOf(100L), converted.get(new KafkaTopicPartition(false, "my-topic", 0))); + Assert.assertEquals(Long.valueOf(200L), converted.get(new KafkaTopicPartition(false, "my-topic", 1))); + } + + @Test + public void testConvertBoundedConfigMapWithMixedValues() + { + TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("my-topic:0", 100); + rawMap.put("my-topic:1", "200"); + rawMap.put("my-topic:2", 300L); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(3, converted.size()); + Assert.assertEquals(Long.valueOf(100L), converted.get(new KafkaTopicPartition(false, "my-topic", 0))); + Assert.assertEquals(Long.valueOf(200L), converted.get(new KafkaTopicPartition(false, "my-topic", 1))); + Assert.assertEquals(Long.valueOf(300L), converted.get(new KafkaTopicPartition(false, "my-topic", 2))); + } + + /** + * Minimal testable subclass that exposes protected methods for testing. + */ + private static class TestableKafkaSupervisor extends KafkaSupervisor + { + public TestableKafkaSupervisor() + { + super( + null, + null, + null, + null, + null, + null, + null + ); + } + + @Override + public KafkaTopicPartition createPartitionIdFromString(String partitionIdString) + { + return super.createPartitionIdFromString(partitionIdString); + } + + @Override + public Long createSequenceOffsetFromObject(Object offsetObj) + { + return super.createSequenceOffsetFromObject(offsetObj); + } + + @Override + public boolean isOffsetAtOrBeyond(Long current, Long target) + { + return super.isOffsetAtOrBeyond(current, target); + } + + public Map convertBoundedConfigMap(Map rawMap) + { + Map result = new HashMap<>(); + for (Map.Entry entry : rawMap.entrySet()) { + KafkaTopicPartition partition = createPartitionIdFromString(entry.getKey().toString()); + Long offset = createSequenceOffsetFromObject(entry.getValue()); + result.put(partition, offset); + } + return result; + } + } +} 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..79c7804991dc 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 @@ -27,6 +27,7 @@ import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; import org.apache.druid.indexing.kafka.KafkaRecordSupplier; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; import org.apache.druid.indexing.seekablestream.supervisor.LagAggregator; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.LagBasedAutoScalerConfig; @@ -442,4 +443,131 @@ public void testIdleConfigSerde() throws JsonProcessingException Assert.assertTrue(kafkaSupervisorIOConfig1.getIdleConfig().isEnabled()); Assert.assertEquals(Long.valueOf(600000), kafkaSupervisorIOConfig1.getIdleConfig().getInactiveAfterMillis()); } + + @Test + public void testBoundedModeSerdeWithIntegerOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kafka\",\n" + + " \"topic\": \"my-topic\",\n" + + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"0\": 100, \"1\": 200},\n" + + " \"endSequenceNumbers\": {\"0\": 500, \"1\": 600}\n" + + " }\n" + + "}"; + + KafkaSupervisorIOConfig config = mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } + + @Test + public void testBoundedModeSerdeWithStringOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kafka\",\n" + + " \"topic\": \"my-topic\",\n" + + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"0\": \"100\", \"1\": \"200\"},\n" + + " \"endSequenceNumbers\": {\"0\": \"500\", \"1\": \"600\"}\n" + + " }\n" + + "}"; + + KafkaSupervisorIOConfig config = mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } + + @Test + public void testBoundedModeSerdeWithMixedOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kafka\",\n" + + " \"topic\": \"my-topic\",\n" + + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"},\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"0\": 100, \"1\": \"200\"},\n" + + " \"endSequenceNumbers\": {\"0\": 500, \"1\": \"600\"}\n" + + " }\n" + + "}"; + + KafkaSupervisorIOConfig config = mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + } + + @Test + public void testUnboundedModeByDefault() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kafka\",\n" + + " \"topic\": \"my-topic\",\n" + + " \"consumerProperties\": {\"bootstrap.servers\":\"localhost:9092\"}\n" + + "}"; + + KafkaSupervisorIOConfig config = mapper.readValue(jsonStr, KafkaSupervisorIOConfig.class); + + Assert.assertFalse(config.isBounded()); + Assert.assertNull(config.getBoundedStreamConfig()); + } + + @Test + public void testBoundedModeRoundTrip() throws Exception + { + final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); + consumerProperties.put("bootstrap.servers", "localhost:8082"); + + Map startOffsets = new HashMap<>(); + startOffsets.put("0", 100); + startOffsets.put("1", 200); + + Map endOffsets = new HashMap<>(); + endOffsets.put("0", 500); + endOffsets.put("1", 600); + + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + KafkaSupervisorIOConfig original = new KafkaSupervisorIOConfig( + "test-topic", + null, + null, + 1, + 1, + new Period("PT1H"), + consumerProperties, + null, + LagAggregator.DEFAULT, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + null, + null, + null, + null, + false, + null, + boundedConfig + ); + + String json = mapper.writeValueAsString(original); + KafkaSupervisorIOConfig deserialized = mapper.readValue(json, KafkaSupervisorIOConfig.class); + + Assert.assertTrue(deserialized.isBounded()); + Assert.assertNotNull(deserialized.getBoundedStreamConfig()); + Assert.assertEquals(2, deserialized.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, deserialized.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } } 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..daef2fe13d00 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 @@ -5477,6 +5477,98 @@ public void testComputeUnassignedServerPriorities_whenMultipleReplicasPerPriorit Assert.assertEquals(List.of(20, 20), supervisor.computeUnassignedServerPriorities(taskGroup3, 2)); } + @Test + public void testBoundedModeCreateTasksWithCorrectOffsets() throws JsonProcessingException + { + Map startOffsets = ImmutableMap.of( + topic + ":0", 100, + topic + ":1", 200, + topic + ":2", 300 + ); + Map endOffsets = ImmutableMap.of( + topic + ":0", 500, + topic + ":1", 600, + topic + ":2", 700 + ); + + final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); + consumerProperties.put("bootstrap.servers", kafkaHost); + + final KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( + topic, + null, + INPUT_FORMAT, + 1, + 1, + new Period("PT1H"), + consumerProperties, + null, + null, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + null, + null, + null, + null, + true, + null, + new org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig(startOffsets, endOffsets) + ); + + Assert.assertTrue(kafkaSupervisorIOConfig.isBounded()); + + final KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(null, null); + final KafkaSupervisorSpec spec = new KafkaSupervisorSpec( + null, + null, + dataSchema, + KafkaSupervisorTuningConfig.defaultConfig(), + kafkaSupervisorIOConfig, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + new SupervisorStateManagerConfig() + ); + + supervisor = new TestableKafkaSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + spec, + rowIngestionMetersFactory + ); + + // Test type conversion methods + KafkaTopicPartition partition0 = supervisor.createPartitionIdFromString(topic + ":0"); + Assert.assertEquals(topic, partition0.topic()); + Assert.assertEquals(0, partition0.partition()); + + Long offset = supervisor.createSequenceOffsetFromObject(100); + Assert.assertEquals(Long.valueOf(100L), offset); + + offset = supervisor.createSequenceOffsetFromObject("200"); + Assert.assertEquals(Long.valueOf(200L), offset); + + // Test offset comparison + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(500L, 100L)); + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(100L, 100L)); + Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); + } + private void addSomeEvents(int numEventsPerPartition) throws Exception { // create topic manually diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java new file mode 100644 index 000000000000..8c62e39e3fb9 --- /dev/null +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java @@ -0,0 +1,202 @@ +/* + * 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.kinesis.supervisor; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class KinesisSupervisorBoundedModeTest +{ + @Test + public void testCreatePartitionIdFromString() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + String partition = supervisor.createPartitionIdFromString("shardId-000000000000"); + + Assert.assertEquals("shardId-000000000000", partition); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithString() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + String offset = supervisor.createSequenceOffsetFromObject("49590338271490256608559692538361571095921575989136588898"); + + Assert.assertEquals("49590338271490256608559692538361571095921575989136588898", offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithInteger() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + String offset = supervisor.createSequenceOffsetFromObject(100); + + Assert.assertEquals("100", offset); + } + + @Test + public void testCreateSequenceOffsetFromObjectWithLong() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + String offset = supervisor.createSequenceOffsetFromObject(100L); + + Assert.assertEquals("100", offset); + } + + @Test + public void testIsOffsetAtOrBeyondEqual() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + Assert.assertTrue(supervisor.isOffsetAtOrBeyond("49590338271490256608559692538361571095921575989136588898", "49590338271490256608559692538361571095921575989136588898")); + } + + @Test + public void testIsOffsetAtOrBeyondGreater() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + Assert.assertTrue(supervisor.isOffsetAtOrBeyond("49590338271512257353759162668991891722121171891717232706", "49590338271490256608559692538361571095921575989136588898")); + } + + @Test + public void testIsOffsetAtOrBeyondLess() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + Assert.assertFalse(supervisor.isOffsetAtOrBeyond("49590338271490256608559692538361571095921575989136588898", "49590338271512257353759162668991891722121171891717232706")); + } + + @Test + public void testIsOffsetAtOrBeyondWithSimpleStrings() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + Assert.assertTrue(supervisor.isOffsetAtOrBeyond("200", "100")); + Assert.assertFalse(supervisor.isOffsetAtOrBeyond("100", "200")); + Assert.assertTrue(supervisor.isOffsetAtOrBeyond("100", "100")); + } + + @Test + public void testConvertBoundedConfigMapWithStringValues() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("shardId-000000000000", "49590338271490256608559692538361571095921575989136588898"); + rawMap.put("shardId-000000000001", "49590338271512257353759162668991891722121171891717232706"); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(2, converted.size()); + Assert.assertEquals("49590338271490256608559692538361571095921575989136588898", converted.get("shardId-000000000000")); + Assert.assertEquals("49590338271512257353759162668991891722121171891717232706", converted.get("shardId-000000000001")); + } + + @Test + public void testConvertBoundedConfigMapWithNumericValues() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("shardId-000000000000", 100); + rawMap.put("shardId-000000000001", 200L); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(2, converted.size()); + Assert.assertEquals("100", converted.get("shardId-000000000000")); + Assert.assertEquals("200", converted.get("shardId-000000000001")); + } + + @Test + public void testConvertBoundedConfigMapWithMixedValues() + { + TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); + + Map rawMap = new HashMap<>(); + rawMap.put("shardId-000000000000", "49590338271490256608559692538361571095921575989136588898"); + rawMap.put("shardId-000000000001", 100); + rawMap.put("shardId-000000000002", 200L); + + Map converted = supervisor.convertBoundedConfigMap(rawMap); + + Assert.assertEquals(3, converted.size()); + Assert.assertEquals("49590338271490256608559692538361571095921575989136588898", converted.get("shardId-000000000000")); + Assert.assertEquals("100", converted.get("shardId-000000000001")); + Assert.assertEquals("200", converted.get("shardId-000000000002")); + } + + /** + * Minimal testable subclass that exposes protected methods for testing. + */ + private static class TestableKinesisSupervisor extends KinesisSupervisor + { + public TestableKinesisSupervisor() + { + super( + null, + null, + null, + null, + null, + null, + null, + null + ); + } + + @Override + public String createPartitionIdFromString(String partitionIdString) + { + return super.createPartitionIdFromString(partitionIdString); + } + + @Override + public String createSequenceOffsetFromObject(Object offsetObj) + { + return super.createSequenceOffsetFromObject(offsetObj); + } + + @Override + public boolean isOffsetAtOrBeyond(String current, String target) + { + return super.isOffsetAtOrBeyond(current, target); + } + + public Map convertBoundedConfigMap(Map rawMap) + { + Map result = new HashMap<>(); + for (Map.Entry entry : rawMap.entrySet()) { + String partition = createPartitionIdFromString(entry.getKey().toString()); + String offset = createSequenceOffsetFromObject(entry.getValue()); + result.put(partition, offset); + } + return result; + } + } +} diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java index aa922b008a55..7683cacb97f3 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; import org.apache.druid.indexing.kinesis.KinesisRegion; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.hamcrest.CoreMatchers; import org.joda.time.Duration; @@ -132,4 +133,76 @@ public void testTopicRequired() throws Exception mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); } + @Test + public void testBoundedModeSerdeWithStringOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"shardId-000000000000\": \"49590338271490256608559692538361571095921575989136588898\", \"shardId-000000000001\": \"49590338271512257353759162668991891722121171891717232706\"},\n" + + " \"endSequenceNumbers\": {\"shardId-000000000000\": \"49590338271534258098958632799622211348320767794297876514\", \"shardId-000000000001\": \"49590338271556258844158102930252531974520363696878520322\"}\n" + + " }\n" + + "}"; + + KinesisSupervisorIOConfig config = mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } + + @Test + public void testBoundedModeSerdeWithNumericOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"shardId-000000000000\": 100, \"shardId-000000000001\": 200},\n" + + " \"endSequenceNumbers\": {\"shardId-000000000000\": 500, \"shardId-000000000001\": 600}\n" + + " }\n" + + "}"; + + KinesisSupervisorIOConfig config = mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getBoundedStreamConfig().getEndSequenceNumbers().size()); + } + + @Test + public void testBoundedModeSerdeWithMixedOffsets() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"boundedStreamConfig\": {\n" + + " \"startSequenceNumbers\": {\"shardId-000000000000\": \"49590338271490256608559692538361571095921575989136588898\", \"shardId-000000000001\": 200},\n" + + " \"endSequenceNumbers\": {\"shardId-000000000000\": 500, \"shardId-000000000001\": \"49590338271556258844158102930252531974520363696878520322\"}\n" + + " }\n" + + "}"; + + KinesisSupervisorIOConfig config = mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + } + + @Test + public void testUnboundedModeByDefault() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"kinesis\",\n" + + " \"stream\": \"my-stream\"\n" + + "}"; + + KinesisSupervisorIOConfig config = mapper.readValue(jsonStr, KinesisSupervisorIOConfig.class); + + Assert.assertFalse(config.isBounded()); + Assert.assertNull(config.getBoundedStreamConfig()); + } + } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 9c82671abccb..0dfba54fa07c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -65,6 +65,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.indexing.seekablestream.supervisor.TaskReportData; @@ -4732,6 +4733,100 @@ public void test_doesTaskMatchSupervisor() Assert.assertFalse(supervisor.doesTaskMatchSupervisor(differentTaskType)); } + @Test + public void testBoundedModeCreateTasksWithCorrectOffsets() + { + Map startOffsets = ImmutableMap.of( + "shardId-000000000000", "49590338271490256608559692538361571095921575989136588898", + "shardId-000000000001", "49590338271512257353759162668991891722121171891717232706" + ); + Map endOffsets = ImmutableMap.of( + "shardId-000000000000", "49590338271534258098958632799622211348320767794297876514", + "shardId-000000000001", "49590338271556258844158102930252531974520363696878520322" + ); + final KinesisSupervisorIOConfig kinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( + STREAM, + INPUT_FORMAT, + "awsEndpoint", + null, + 1, + 1, + new Period("PT30S"), + null, + new Period("PT30M"), + null, + null, + null, + null, + null, + null, + 0, + null, + null, + null, + true, + null, + new BoundedStreamConfig(startOffsets, endOffsets) + ); + + Assert.assertTrue(kinesisSupervisorIOConfig.isBounded()); + + final KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(null, null); + final KinesisSupervisorSpec spec = new KinesisSupervisorSpec( + null, + null, + dataSchema, + KinesisSupervisorTuningConfig.defaultConfig(), + kinesisSupervisorIOConfig, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + null, + new SupervisorStateManagerConfig() + ); + + supervisor = new TestableKinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + spec, + rowIngestionMetersFactory + ); + + // Test type conversion methods + String shardId = supervisor.createPartitionIdFromString("shardId-000000000000"); + Assert.assertEquals("shardId-000000000000", shardId); + + String offset = supervisor.createSequenceOffsetFromObject("49590338271490256608559692538361571095921575989136588898"); + Assert.assertEquals("49590338271490256608559692538361571095921575989136588898", offset); + + offset = supervisor.createSequenceOffsetFromObject(100); + Assert.assertEquals("100", offset); + + // Test offset comparison (lexicographic) + Assert.assertTrue(supervisor.isOffsetAtOrBeyond( + "49590338271512257353759162668991891722121171891717232706", + "49590338271490256608559692538361571095921575989136588898" + )); + Assert.assertTrue(supervisor.isOffsetAtOrBeyond( + "49590338271490256608559692538361571095921575989136588898", + "49590338271490256608559692538361571095921575989136588898" + )); + Assert.assertFalse(supervisor.isOffsetAtOrBeyond( + "49590338271490256608559692538361571095921575989136588898", + "49590338271512257353759162668991891722121171891717232706" + )); + } + private List testShardMergePhaseOne() throws Exception { supervisorRecordSupplier.assign(EasyMock.anyObject()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java new file mode 100644 index 000000000000..63ec54124227 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java @@ -0,0 +1,175 @@ +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class BoundedStreamConfigTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + + @Test + public void testConstructorWithValidMaps() + { + Map startOffsets = new HashMap<>(); + startOffsets.put("0", 100L); + startOffsets.put("1", 200L); + + Map endOffsets = new HashMap<>(); + endOffsets.put("0", 500L); + endOffsets.put("1", 600L); + + BoundedStreamConfig config = new BoundedStreamConfig(startOffsets, endOffsets); + + Assert.assertEquals(startOffsets, config.getStartSequenceNumbers()); + Assert.assertEquals(endOffsets, config.getEndSequenceNumbers()); + } + + @Test + public void testConstructorWithNullStartSequenceNumbers() + { + Map endOffsets = new HashMap<>(); + endOffsets.put("0", 500L); + + IllegalArgumentException ex = Assert.assertThrows( + IllegalArgumentException.class, + () -> new BoundedStreamConfig(null, endOffsets) + ); + + Assert.assertEquals("startSequenceNumbers cannot be null", ex.getMessage()); + } + + @Test + public void testConstructorWithNullEndSequenceNumbers() + { + Map startOffsets = new HashMap<>(); + startOffsets.put("0", 100L); + + IllegalArgumentException ex = Assert.assertThrows( + IllegalArgumentException.class, + () -> new BoundedStreamConfig(startOffsets, null) + ); + + Assert.assertEquals("endSequenceNumbers cannot be null", ex.getMessage()); + } + + @Test + public void testConstructorWithEmptyStartSequenceNumbers() + { + Map startOffsets = new HashMap<>(); + Map endOffsets = new HashMap<>(); + endOffsets.put("0", 500L); + + IllegalArgumentException ex = Assert.assertThrows( + IllegalArgumentException.class, + () -> new BoundedStreamConfig(startOffsets, endOffsets) + ); + + Assert.assertEquals("startSequenceNumbers cannot be empty", ex.getMessage()); + } + + @Test + public void testConstructorWithEmptyEndSequenceNumbers() + { + Map startOffsets = new HashMap<>(); + startOffsets.put("0", 100L); + Map endOffsets = new HashMap<>(); + + IllegalArgumentException ex = Assert.assertThrows( + IllegalArgumentException.class, + () -> new BoundedStreamConfig(startOffsets, endOffsets) + ); + + Assert.assertEquals("endSequenceNumbers cannot be empty", ex.getMessage()); + } + + @Test + public void testSerializationDeserialization() throws Exception + { + Map startOffsets = new HashMap<>(); + startOffsets.put(0, 100); + startOffsets.put(1, 200); + + Map endOffsets = new HashMap<>(); + endOffsets.put(0, 500); + endOffsets.put(1, 600); + + BoundedStreamConfig config = new BoundedStreamConfig(startOffsets, endOffsets); + + String json = mapper.writeValueAsString(config); + BoundedStreamConfig deserialized = mapper.readValue(json, BoundedStreamConfig.class); + + Assert.assertEquals(config.getStartSequenceNumbers(), deserialized.getStartSequenceNumbers()); + Assert.assertEquals(config.getEndSequenceNumbers(), deserialized.getEndSequenceNumbers()); + } + + @Test + public void testDeserializationWithIntegerValues() throws Exception + { + String json = "{" + + "\"startSequenceNumbers\": {\"0\": 100, \"1\": 200}," + + "\"endSequenceNumbers\": {\"0\": 500, \"1\": 600}" + + "}"; + + BoundedStreamConfig config = mapper.readValue(json, BoundedStreamConfig.class); + + Assert.assertNotNull(config.getStartSequenceNumbers()); + Assert.assertNotNull(config.getEndSequenceNumbers()); + Assert.assertEquals(2, config.getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getEndSequenceNumbers().size()); + } + + @Test + public void testDeserializationWithStringValues() throws Exception + { + String json = "{" + + "\"startSequenceNumbers\": {\"0\": \"100\", \"1\": \"200\"}," + + "\"endSequenceNumbers\": {\"0\": \"500\", \"1\": \"600\"}" + + "}"; + + BoundedStreamConfig config = mapper.readValue(json, BoundedStreamConfig.class); + + Assert.assertNotNull(config.getStartSequenceNumbers()); + Assert.assertNotNull(config.getEndSequenceNumbers()); + Assert.assertEquals(2, config.getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getEndSequenceNumbers().size()); + } + + @Test + public void testDeserializationWithMixedTypes() throws Exception + { + String json = "{" + + "\"startSequenceNumbers\": {\"0\": 100, \"1\": \"200\"}," + + "\"endSequenceNumbers\": {\"0\": 500, \"1\": \"600\"}" + + "}"; + + BoundedStreamConfig config = mapper.readValue(json, BoundedStreamConfig.class); + + Assert.assertNotNull(config.getStartSequenceNumbers()); + Assert.assertNotNull(config.getEndSequenceNumbers()); + Assert.assertEquals(2, config.getStartSequenceNumbers().size()); + Assert.assertEquals(2, config.getEndSequenceNumbers().size()); + } +} 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..2c018b18f544 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 @@ -472,4 +472,105 @@ private SeekableStreamSupervisorIOConfig makeSeekableStreamSupervisorIOConfig(@N { }; } + + @Test + public void testBoundedModeWithValidConfig() + { + Map startOffsets = Map.of("0", 100, "1", 200); + Map endOffsets = Map.of("0", 500, "1", 600); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + LagAggregator lagAggregator = mock(LagAggregator.class); + + SeekableStreamSupervisorIOConfig config = new SeekableStreamSupervisorIOConfig( + "stream", + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + lagAggregator, + null, + null, + null, + null, + boundedConfig + ) + { + }; + + Assert.assertTrue(config.isBounded()); + Assert.assertNotNull(config.getBoundedStreamConfig()); + Assert.assertEquals(boundedConfig, config.getBoundedStreamConfig()); + } + + @Test + public void testUnboundedModeByDefault() + { + LagAggregator lagAggregator = mock(LagAggregator.class); + + SeekableStreamSupervisorIOConfig config = new SeekableStreamSupervisorIOConfig( + "stream", + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + lagAggregator, + null, + null, + null, + null, + null + ) + { + }; + + Assert.assertFalse(config.isBounded()); + Assert.assertNull(config.getBoundedStreamConfig()); + } + + @Test + public void testBoundedModeWithNullConfig() + { + LagAggregator lagAggregator = mock(LagAggregator.class); + + SeekableStreamSupervisorIOConfig config = new SeekableStreamSupervisorIOConfig( + "stream", + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + lagAggregator, + null, + null, + null, + null, + null + ) + { + }; + + Assert.assertFalse(config.isBounded()); + Assert.assertNull(config.getBoundedStreamConfig()); + } } diff --git a/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java b/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java index 367577a291dc..043992998f72 100644 --- a/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java +++ b/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java @@ -66,4 +66,88 @@ public void testIdleConfigSerde() Assert.assertTrue(stateManagerConfig.isIdleConfigEnabled()); Assert.assertEquals(60000, stateManagerConfig.getInactiveAfterMillis()); } + + @Test + public void testCompletedStateIsTerminal() + { + stateManagerConfig = new SupervisorStateManagerConfig(); + SupervisorStateManager supervisorStateManager = new SupervisorStateManager( + stateManagerConfig, + false + ); + + // Start in PENDING state + Assert.assertEquals(SupervisorStateManager.BasicState.PENDING, supervisorStateManager.getSupervisorState()); + + // Transition to COMPLETED + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.COMPLETED); + Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); + + // Attempt to transition out of COMPLETED should be ignored + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.RUNNING); + Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); + + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.IDLE); + Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); + + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.PENDING); + Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); + } + + @Test + public void testStoppingStateIsTerminal() + { + stateManagerConfig = new SupervisorStateManagerConfig(); + SupervisorStateManager supervisorStateManager = new SupervisorStateManager( + stateManagerConfig, + false + ); + + // Start in PENDING state + Assert.assertEquals(SupervisorStateManager.BasicState.PENDING, supervisorStateManager.getSupervisorState()); + + // Transition to STOPPING + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.STOPPING); + Assert.assertEquals(SupervisorStateManager.BasicState.STOPPING, supervisorStateManager.getSupervisorState()); + + // Attempt to transition out of STOPPING should be ignored + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.RUNNING); + Assert.assertEquals(SupervisorStateManager.BasicState.STOPPING, supervisorStateManager.getSupervisorState()); + + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.IDLE); + Assert.assertEquals(SupervisorStateManager.BasicState.STOPPING, supervisorStateManager.getSupervisorState()); + + // Cannot transition to COMPLETED from STOPPING + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.COMPLETED); + Assert.assertEquals(SupervisorStateManager.BasicState.STOPPING, supervisorStateManager.getSupervisorState()); + } + + @Test + public void testCompletedStateIsHealthy() + { + stateManagerConfig = new SupervisorStateManagerConfig(); + SupervisorStateManager supervisorStateManager = new SupervisorStateManager( + stateManagerConfig, + false + ); + + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.COMPLETED); + + Assert.assertTrue(supervisorStateManager.isHealthy()); + Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); + } + + @Test + public void testCompletedStateIsNotFirstRunOnly() + { + stateManagerConfig = new SupervisorStateManagerConfig(); + SupervisorStateManager supervisorStateManager = new SupervisorStateManager( + stateManagerConfig, + false + ); + + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.COMPLETED); + + Assert.assertFalse(SupervisorStateManager.BasicState.COMPLETED.isFirstRunOnly()); + } } From 300ebe3233f9f07764167631443113eab70de1d1 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 24 Apr 2026 11:10:45 -0700 Subject: [PATCH 04/52] Fix BoundedStreamConfigTest --- .../supervisor/BoundedStreamConfigTest.java | 42 +++++++++++-------- 1 file changed, 25 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java index 63ec54124227..9c86b4d9a6d7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java @@ -53,12 +53,12 @@ public void testConstructorWithNullStartSequenceNumbers() Map endOffsets = new HashMap<>(); endOffsets.put("0", 500L); - IllegalArgumentException ex = Assert.assertThrows( - IllegalArgumentException.class, + NullPointerException ex = Assert.assertThrows( + NullPointerException.class, () -> new BoundedStreamConfig(null, endOffsets) ); - Assert.assertEquals("startSequenceNumbers cannot be null", ex.getMessage()); + Assert.assertTrue(ex.getMessage().contains("startSequenceNumbers")); } @Test @@ -67,12 +67,12 @@ public void testConstructorWithNullEndSequenceNumbers() Map startOffsets = new HashMap<>(); startOffsets.put("0", 100L); - IllegalArgumentException ex = Assert.assertThrows( - IllegalArgumentException.class, + NullPointerException ex = Assert.assertThrows( + NullPointerException.class, () -> new BoundedStreamConfig(startOffsets, null) ); - Assert.assertEquals("endSequenceNumbers cannot be null", ex.getMessage()); + Assert.assertTrue(ex.getMessage().contains("endSequenceNumbers")); } @Test @@ -87,42 +87,50 @@ public void testConstructorWithEmptyStartSequenceNumbers() () -> new BoundedStreamConfig(startOffsets, endOffsets) ); - Assert.assertEquals("startSequenceNumbers cannot be empty", ex.getMessage()); + Assert.assertTrue(ex.getMessage().contains("startSequenceNumbers cannot be empty")); } @Test - public void testConstructorWithEmptyEndSequenceNumbers() + public void testConstructorWithMismatchedPartitions() { Map startOffsets = new HashMap<>(); startOffsets.put("0", 100L); Map endOffsets = new HashMap<>(); + endOffsets.put("1", 500L); IllegalArgumentException ex = Assert.assertThrows( IllegalArgumentException.class, () -> new BoundedStreamConfig(startOffsets, endOffsets) ); - Assert.assertEquals("endSequenceNumbers cannot be empty", ex.getMessage()); + Assert.assertTrue(ex.getMessage().contains("must have matching partition sets")); } @Test public void testSerializationDeserialization() throws Exception { - Map startOffsets = new HashMap<>(); - startOffsets.put(0, 100); - startOffsets.put(1, 200); + Map startOffsets = new HashMap<>(); + startOffsets.put("0", 100); + startOffsets.put("1", 200); - Map endOffsets = new HashMap<>(); - endOffsets.put(0, 500); - endOffsets.put(1, 600); + Map endOffsets = new HashMap<>(); + endOffsets.put("0", 500); + endOffsets.put("1", 600); BoundedStreamConfig config = new BoundedStreamConfig(startOffsets, endOffsets); String json = mapper.writeValueAsString(config); BoundedStreamConfig deserialized = mapper.readValue(json, BoundedStreamConfig.class); - Assert.assertEquals(config.getStartSequenceNumbers(), deserialized.getStartSequenceNumbers()); - Assert.assertEquals(config.getEndSequenceNumbers(), deserialized.getEndSequenceNumbers()); + // Check sizes + Assert.assertEquals(2, deserialized.getStartSequenceNumbers().size()); + Assert.assertEquals(2, deserialized.getEndSequenceNumbers().size()); + + // Check that deserialized maps contain expected values (keys will be Strings after deserialization) + Assert.assertEquals(100, deserialized.getStartSequenceNumbers().get("0")); + Assert.assertEquals(200, deserialized.getStartSequenceNumbers().get("1")); + Assert.assertEquals(500, deserialized.getEndSequenceNumbers().get("0")); + Assert.assertEquals(600, deserialized.getEndSequenceNumbers().get("1")); } @Test From 9af972979f528b08d8b8aa8f69023a0d920e119e Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 24 Apr 2026 11:11:42 -0700 Subject: [PATCH 05/52] Remove unused import --- .../kinesis/supervisor/KinesisSupervisorIOConfigTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java index 7683cacb97f3..a93bdf6871bc 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfigTest.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; import org.apache.druid.indexing.kinesis.KinesisRegion; -import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.hamcrest.CoreMatchers; import org.joda.time.Duration; From e0ffef671c6717acc5f95fbcae0b69ae06753053 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 24 Apr 2026 11:33:38 -0700 Subject: [PATCH 06/52] Remove unneeded tests --- ...RabbitStreamSupervisorBoundedModeTest.java | 204 ----------------- .../KafkaSupervisorBoundedModeTest.java | 206 ------------------ .../KinesisSupervisorBoundedModeTest.java | 202 ----------------- 3 files changed, 612 deletions(-) delete mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java delete mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java delete mode 100644 extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java deleted file mode 100644 index 81420e7fe91f..000000000000 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorBoundedModeTest.java +++ /dev/null @@ -1,204 +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.rabbitstream.supervisor; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -public class RabbitStreamSupervisorBoundedModeTest -{ - @Test - public void testCreatePartitionIdFromString() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - String partition = supervisor.createPartitionIdFromString("queue-0"); - - Assert.assertEquals("queue-0", partition); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithInteger() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Long offset = supervisor.createSequenceOffsetFromObject(100); - - Assert.assertEquals(Long.valueOf(100L), offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithLong() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Long offset = supervisor.createSequenceOffsetFromObject(100L); - - Assert.assertEquals(Long.valueOf(100L), offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithString() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Long offset = supervisor.createSequenceOffsetFromObject("100"); - - Assert.assertEquals(Long.valueOf(100L), offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithInvalidType() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - IllegalArgumentException ex = Assert.assertThrows( - IllegalArgumentException.class, - () -> supervisor.createSequenceOffsetFromObject(new Object()) - ); - - Assert.assertTrue(ex.getMessage().contains("Cannot convert")); - } - - @Test - public void testIsOffsetAtOrBeyondEqual() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Assert.assertTrue(supervisor.isOffsetAtOrBeyond(100L, 100L)); - } - - @Test - public void testIsOffsetAtOrBeyondGreater() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Assert.assertTrue(supervisor.isOffsetAtOrBeyond(200L, 100L)); - } - - @Test - public void testIsOffsetAtOrBeyondLess() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); - } - - @Test - public void testConvertBoundedConfigMapWithIntegerValues() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("queue-0", 100); - rawMap.put("queue-1", 200); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(2, converted.size()); - Assert.assertEquals(Long.valueOf(100L), converted.get("queue-0")); - Assert.assertEquals(Long.valueOf(200L), converted.get("queue-1")); - } - - @Test - public void testConvertBoundedConfigMapWithStringValues() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("queue-0", "100"); - rawMap.put("queue-1", "200"); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(2, converted.size()); - Assert.assertEquals(Long.valueOf(100L), converted.get("queue-0")); - Assert.assertEquals(Long.valueOf(200L), converted.get("queue-1")); - } - - @Test - public void testConvertBoundedConfigMapWithMixedValues() - { - TestableRabbitStreamSupervisor supervisor = new TestableRabbitStreamSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("queue-0", 100); - rawMap.put("queue-1", "200"); - rawMap.put("queue-2", 300L); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(3, converted.size()); - Assert.assertEquals(Long.valueOf(100L), converted.get("queue-0")); - Assert.assertEquals(Long.valueOf(200L), converted.get("queue-1")); - Assert.assertEquals(Long.valueOf(300L), converted.get("queue-2")); - } - - /** - * Minimal testable subclass that exposes protected methods for testing. - */ - private static class TestableRabbitStreamSupervisor extends RabbitStreamSupervisor - { - public TestableRabbitStreamSupervisor() - { - super( - null, - null, - null, - null, - null, - null, - null - ); - } - - @Override - public String createPartitionIdFromString(String partitionIdString) - { - return super.createPartitionIdFromString(partitionIdString); - } - - @Override - public Long createSequenceOffsetFromObject(Object offsetObj) - { - return super.createSequenceOffsetFromObject(offsetObj); - } - - @Override - public boolean isOffsetAtOrBeyond(Long current, Long target) - { - return super.isOffsetAtOrBeyond(current, target); - } - - public Map convertBoundedConfigMap(Map rawMap) - { - Map result = new HashMap<>(); - for (Map.Entry entry : rawMap.entrySet()) { - String partition = createPartitionIdFromString(entry.getKey().toString()); - Long offset = createSequenceOffsetFromObject(entry.getValue()); - result.put(partition, offset); - } - return result; - } - } -} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java deleted file mode 100644 index b654a68dadd6..000000000000 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorBoundedModeTest.java +++ /dev/null @@ -1,206 +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.kafka.supervisor; - -import org.apache.druid.data.input.kafka.KafkaTopicPartition; -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -public class KafkaSupervisorBoundedModeTest -{ - @Test - public void testCreatePartitionIdFromString() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - KafkaTopicPartition partition = supervisor.createPartitionIdFromString("my-topic:5"); - - Assert.assertEquals("my-topic", partition.topic()); - Assert.assertEquals(5, partition.partition()); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithInteger() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Long offset = supervisor.createSequenceOffsetFromObject(100); - - Assert.assertEquals(Long.valueOf(100L), offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithLong() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Long offset = supervisor.createSequenceOffsetFromObject(100L); - - Assert.assertEquals(Long.valueOf(100L), offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithString() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Long offset = supervisor.createSequenceOffsetFromObject("100"); - - Assert.assertEquals(Long.valueOf(100L), offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithInvalidType() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - IllegalArgumentException ex = Assert.assertThrows( - IllegalArgumentException.class, - () -> supervisor.createSequenceOffsetFromObject(new Object()) - ); - - Assert.assertTrue(ex.getMessage().contains("Cannot convert")); - } - - @Test - public void testIsOffsetAtOrBeyondEqual() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Assert.assertTrue(supervisor.isOffsetAtOrBeyond(100L, 100L)); - } - - @Test - public void testIsOffsetAtOrBeyondGreater() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Assert.assertTrue(supervisor.isOffsetAtOrBeyond(200L, 100L)); - } - - @Test - public void testIsOffsetAtOrBeyondLess() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); - } - - @Test - public void testConvertBoundedConfigMapWithIntegerValues() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("my-topic:0", 100); - rawMap.put("my-topic:1", 200); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(2, converted.size()); - Assert.assertEquals(Long.valueOf(100L), converted.get(new KafkaTopicPartition(false, "my-topic", 0))); - Assert.assertEquals(Long.valueOf(200L), converted.get(new KafkaTopicPartition(false, "my-topic", 1))); - } - - @Test - public void testConvertBoundedConfigMapWithStringValues() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("my-topic:0", "100"); - rawMap.put("my-topic:1", "200"); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(2, converted.size()); - Assert.assertEquals(Long.valueOf(100L), converted.get(new KafkaTopicPartition(false, "my-topic", 0))); - Assert.assertEquals(Long.valueOf(200L), converted.get(new KafkaTopicPartition(false, "my-topic", 1))); - } - - @Test - public void testConvertBoundedConfigMapWithMixedValues() - { - TestableKafkaSupervisor supervisor = new TestableKafkaSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("my-topic:0", 100); - rawMap.put("my-topic:1", "200"); - rawMap.put("my-topic:2", 300L); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(3, converted.size()); - Assert.assertEquals(Long.valueOf(100L), converted.get(new KafkaTopicPartition(false, "my-topic", 0))); - Assert.assertEquals(Long.valueOf(200L), converted.get(new KafkaTopicPartition(false, "my-topic", 1))); - Assert.assertEquals(Long.valueOf(300L), converted.get(new KafkaTopicPartition(false, "my-topic", 2))); - } - - /** - * Minimal testable subclass that exposes protected methods for testing. - */ - private static class TestableKafkaSupervisor extends KafkaSupervisor - { - public TestableKafkaSupervisor() - { - super( - null, - null, - null, - null, - null, - null, - null - ); - } - - @Override - public KafkaTopicPartition createPartitionIdFromString(String partitionIdString) - { - return super.createPartitionIdFromString(partitionIdString); - } - - @Override - public Long createSequenceOffsetFromObject(Object offsetObj) - { - return super.createSequenceOffsetFromObject(offsetObj); - } - - @Override - public boolean isOffsetAtOrBeyond(Long current, Long target) - { - return super.isOffsetAtOrBeyond(current, target); - } - - public Map convertBoundedConfigMap(Map rawMap) - { - Map result = new HashMap<>(); - for (Map.Entry entry : rawMap.entrySet()) { - KafkaTopicPartition partition = createPartitionIdFromString(entry.getKey().toString()); - Long offset = createSequenceOffsetFromObject(entry.getValue()); - result.put(partition, offset); - } - return result; - } - } -} diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java deleted file mode 100644 index 8c62e39e3fb9..000000000000 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorBoundedModeTest.java +++ /dev/null @@ -1,202 +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.kinesis.supervisor; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -public class KinesisSupervisorBoundedModeTest -{ - @Test - public void testCreatePartitionIdFromString() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - String partition = supervisor.createPartitionIdFromString("shardId-000000000000"); - - Assert.assertEquals("shardId-000000000000", partition); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithString() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - String offset = supervisor.createSequenceOffsetFromObject("49590338271490256608559692538361571095921575989136588898"); - - Assert.assertEquals("49590338271490256608559692538361571095921575989136588898", offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithInteger() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - String offset = supervisor.createSequenceOffsetFromObject(100); - - Assert.assertEquals("100", offset); - } - - @Test - public void testCreateSequenceOffsetFromObjectWithLong() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - String offset = supervisor.createSequenceOffsetFromObject(100L); - - Assert.assertEquals("100", offset); - } - - @Test - public void testIsOffsetAtOrBeyondEqual() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - Assert.assertTrue(supervisor.isOffsetAtOrBeyond("49590338271490256608559692538361571095921575989136588898", "49590338271490256608559692538361571095921575989136588898")); - } - - @Test - public void testIsOffsetAtOrBeyondGreater() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - Assert.assertTrue(supervisor.isOffsetAtOrBeyond("49590338271512257353759162668991891722121171891717232706", "49590338271490256608559692538361571095921575989136588898")); - } - - @Test - public void testIsOffsetAtOrBeyondLess() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - Assert.assertFalse(supervisor.isOffsetAtOrBeyond("49590338271490256608559692538361571095921575989136588898", "49590338271512257353759162668991891722121171891717232706")); - } - - @Test - public void testIsOffsetAtOrBeyondWithSimpleStrings() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - Assert.assertTrue(supervisor.isOffsetAtOrBeyond("200", "100")); - Assert.assertFalse(supervisor.isOffsetAtOrBeyond("100", "200")); - Assert.assertTrue(supervisor.isOffsetAtOrBeyond("100", "100")); - } - - @Test - public void testConvertBoundedConfigMapWithStringValues() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("shardId-000000000000", "49590338271490256608559692538361571095921575989136588898"); - rawMap.put("shardId-000000000001", "49590338271512257353759162668991891722121171891717232706"); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(2, converted.size()); - Assert.assertEquals("49590338271490256608559692538361571095921575989136588898", converted.get("shardId-000000000000")); - Assert.assertEquals("49590338271512257353759162668991891722121171891717232706", converted.get("shardId-000000000001")); - } - - @Test - public void testConvertBoundedConfigMapWithNumericValues() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("shardId-000000000000", 100); - rawMap.put("shardId-000000000001", 200L); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(2, converted.size()); - Assert.assertEquals("100", converted.get("shardId-000000000000")); - Assert.assertEquals("200", converted.get("shardId-000000000001")); - } - - @Test - public void testConvertBoundedConfigMapWithMixedValues() - { - TestableKinesisSupervisor supervisor = new TestableKinesisSupervisor(); - - Map rawMap = new HashMap<>(); - rawMap.put("shardId-000000000000", "49590338271490256608559692538361571095921575989136588898"); - rawMap.put("shardId-000000000001", 100); - rawMap.put("shardId-000000000002", 200L); - - Map converted = supervisor.convertBoundedConfigMap(rawMap); - - Assert.assertEquals(3, converted.size()); - Assert.assertEquals("49590338271490256608559692538361571095921575989136588898", converted.get("shardId-000000000000")); - Assert.assertEquals("100", converted.get("shardId-000000000001")); - Assert.assertEquals("200", converted.get("shardId-000000000002")); - } - - /** - * Minimal testable subclass that exposes protected methods for testing. - */ - private static class TestableKinesisSupervisor extends KinesisSupervisor - { - public TestableKinesisSupervisor() - { - super( - null, - null, - null, - null, - null, - null, - null, - null - ); - } - - @Override - public String createPartitionIdFromString(String partitionIdString) - { - return super.createPartitionIdFromString(partitionIdString); - } - - @Override - public String createSequenceOffsetFromObject(Object offsetObj) - { - return super.createSequenceOffsetFromObject(offsetObj); - } - - @Override - public boolean isOffsetAtOrBeyond(String current, String target) - { - return super.isOffsetAtOrBeyond(current, target); - } - - public Map convertBoundedConfigMap(Map rawMap) - { - Map result = new HashMap<>(); - for (Map.Entry entry : rawMap.entrySet()) { - String partition = createPartitionIdFromString(entry.getKey().toString()); - String offset = createSequenceOffsetFromObject(entry.getValue()); - result.put(partition, offset); - } - return result; - } - } -} From 162e1f339be911d5bb6a51530afe98461d2129d8 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 24 Apr 2026 13:15:59 -0700 Subject: [PATCH 07/52] Unit test fix --- .../druid/indexing/kafka/supervisor/KafkaSupervisorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 daef2fe13d00..7c5a0610c8db 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 @@ -5554,7 +5554,7 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() throws JsonProcessing // Test type conversion methods KafkaTopicPartition partition0 = supervisor.createPartitionIdFromString(topic + ":0"); - Assert.assertEquals(topic, partition0.topic()); + Assert.assertEquals(topic, partition0.topic().get()); Assert.assertEquals(0, partition0.partition()); Long offset = supervisor.createSequenceOffsetFromObject(100); From 3ea2b0b394dc3160cf3198655773a767a8911b3a Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 24 Apr 2026 14:23:08 -0700 Subject: [PATCH 08/52] Fix import and add coverage for RabbitStreamSupervisor --- .../RabbitStreamSupervisorTest.java | 30 +++++++++++++++++++ .../kafka/supervisor/KafkaSupervisorTest.java | 3 +- 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index b9ab01e13248..19a5cb86d298 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -504,5 +504,35 @@ public void testBoundedModeConfiguration() Assert.assertNotNull(rabbitSupervisorIOConfig.getBoundedStreamConfig()); Assert.assertEquals(2, rabbitSupervisorIOConfig.getBoundedStreamConfig().getStartSequenceNumbers().size()); Assert.assertEquals(2, rabbitSupervisorIOConfig.getBoundedStreamConfig().getEndSequenceNumbers().size()); + + // Create supervisor to test type conversion methods + supervisor = getSupervisor( + "supervisorId", + 1, + 1, + false, + "PT1H", + null, + null, + dataSchema, + tuningConfig + ); + + // Test createPartitionIdFromString + String queueName = supervisor.createPartitionIdFromString("queue-0"); + Assert.assertEquals("queue-0", queueName); + + // Test createSequenceOffsetFromObject with Integer + Long offset = supervisor.createSequenceOffsetFromObject(100); + Assert.assertEquals(Long.valueOf(100L), offset); + + // Test createSequenceOffsetFromObject with String + offset = supervisor.createSequenceOffsetFromObject("200"); + Assert.assertEquals(Long.valueOf(200L), offset); + + // Test isOffsetAtOrBeyond + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(500L, 100L)); + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(100L, 100L)); + Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); } } 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 7c5a0610c8db..1f3bedae78fb 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 @@ -70,6 +70,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; @@ -5517,7 +5518,7 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() throws JsonProcessing null, true, null, - new org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig(startOffsets, endOffsets) + new BoundedStreamConfig(startOffsets, endOffsets) ); Assert.assertTrue(kafkaSupervisorIOConfig.isBounded()); From 8e3e81cf2c52e30ef7c4b9cf3fab4dd998c6a9ab Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 24 Apr 2026 15:11:46 -0700 Subject: [PATCH 09/52] Test coverage for validateBoundedStreamConfig --- .../SeekableStreamSupervisorSpecTest.java | 145 ++++++++++++++++++ 1 file changed, 145 insertions(+) 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 0efad16bc276..b914da4ebb17 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 @@ -1171,6 +1171,151 @@ public String getSource() originalSpec.validateSpecUpdateTo(proposedSpecSameSource); } + @Test + public void test_validateBoundedStreamConfig_WithValidConfig() + { + mockIngestionSchema(); + + Map startOffsets = ImmutableMap.of("0", 100, "1", 200); + Map endOffsets = ImmutableMap.of("0", 500, "1", 600); + BoundedStreamConfig validConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + SeekableStreamSupervisorIOConfig ioConfigWithValidBounded = new SeekableStreamSupervisorIOConfig( + "stream", + new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), false, false, false), + 1, + 1, + new Period("PT1H"), + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + null, + LagAggregator.DEFAULT, + null, + null, + null, + null, + validConfig + ) + { + }; + + SeekableStreamSupervisorIngestionSpec ingestionWithValidBounded = EasyMock.mock(SeekableStreamSupervisorIngestionSpec.class); + EasyMock.expect(ingestionWithValidBounded.getIOConfig()).andReturn(ioConfigWithValidBounded).anyTimes(); + EasyMock.expect(ingestionWithValidBounded.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionWithValidBounded.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionWithValidBounded); + + TestSeekableStreamSupervisorSpec spec = new TestSeekableStreamSupervisorSpec( + ingestionSchema, + ImmutableMap.of("key", "value"), + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig, + supervisor4, + "id1" + ); + + TestSeekableStreamSupervisorSpec specWithValidBounded = new TestSeekableStreamSupervisorSpec( + ingestionWithValidBounded, + ImmutableMap.of("key", "value"), + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig, + supervisor4, + "id1" + ); + + // Should not throw + spec.validateBoundedStreamConfig(specWithValidBounded); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void test_validateBoundedStreamConfig_WithMismatchedPartitions() + { + mockIngestionSchema(); + + // Create a mock BoundedStreamConfig that returns mismatched partition sets + BoundedStreamConfig mismatchedConfig = EasyMock.mock(BoundedStreamConfig.class); + Map startMap = ImmutableMap.of("0", 100L, "1", 200L); + Map endMap = ImmutableMap.of("0", 500L, "2", 600L); + EasyMock.expect(mismatchedConfig.getStartSequenceNumbers()).andStubReturn(startMap); + EasyMock.expect(mismatchedConfig.getEndSequenceNumbers()).andStubReturn(endMap); + EasyMock.replay(mismatchedConfig); + + SeekableStreamSupervisorIOConfig ioConfigWithMismatchedBounded = EasyMock.mock(SeekableStreamSupervisorIOConfig.class); + EasyMock.expect(ioConfigWithMismatchedBounded.isBounded()).andReturn(true).anyTimes(); + EasyMock.expect(ioConfigWithMismatchedBounded.getBoundedStreamConfig()).andReturn(mismatchedConfig).anyTimes(); + EasyMock.replay(ioConfigWithMismatchedBounded); + + SeekableStreamSupervisorIngestionSpec ingestionWithMismatchedBounded = EasyMock.mock(SeekableStreamSupervisorIngestionSpec.class); + EasyMock.expect(ingestionWithMismatchedBounded.getIOConfig()).andReturn(ioConfigWithMismatchedBounded).anyTimes(); + EasyMock.expect(ingestionWithMismatchedBounded.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionWithMismatchedBounded.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionWithMismatchedBounded); + + TestSeekableStreamSupervisorSpec spec = new TestSeekableStreamSupervisorSpec( + ingestionSchema, + ImmutableMap.of("key", "value"), + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig, + supervisor4, + "id1" + ); + + TestSeekableStreamSupervisorSpec specWithMismatchedBounded = new TestSeekableStreamSupervisorSpec( + ingestionWithMismatchedBounded, + ImmutableMap.of("key", "value"), + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig, + supervisor4, + "id1" + ); + + MatcherAssert.assertThat( + assertThrows(DruidException.class, () -> spec.validateBoundedStreamConfig(specWithMismatchedBounded)), + new DruidExceptionMatcher( + DruidException.Persona.USER, + DruidException.Category.INVALID_INPUT, + "invalidInput" + ).expectMessageContains("Bounded stream config has mismatched partitions") + ); + } + @Test public void test_dynamicAllocationNotice_skipsScalingAndEmitsReason_ifTasksArePublishing() throws InterruptedException { From 4bed6580f659959873a01b1ea926bb8c8c96fa26 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 29 Apr 2026 17:45:57 -0700 Subject: [PATCH 10/52] Re-initialize partition group and reset state after reset --- .../supervisor/SeekableStreamSupervisor.java | 26 ++++++++----------- .../supervisor/SupervisorStateManager.java | 21 ++++++++------- 2 files changed, 22 insertions(+), 25 deletions(-) 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 b006e90119d1..30d40a5a1438 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 @@ -1386,13 +1386,7 @@ public void tryInit() // Initialize bounded partitions BEFORE first run if (ioConfig.isBounded()) { - try { - initializeBoundedPartitionGroups(); - } - catch (Exception e) { - log.error(e, "Failed to initialize bounded partition groups"); - throw new RuntimeException(e); - } + initializeBoundedPartitionGroups(); } exec.submit( @@ -1855,6 +1849,10 @@ public TaskGroup addTaskGroupToPendingCompletionTaskGroup( public void runInternal() { try { + if (isBoundedWorkComplete()) { + handleBoundedCompletion(); + return; + } possiblyRegisterListener(); stateManager.maybeSetState(SeekableStreamSupervisorStateManager.SeekableStreamState.CONNECTING_TO_STREAM); @@ -1899,12 +1897,6 @@ public void runInternal() } } - // Check for bounded completion after tasks have been created/managed - if (isBoundedWorkComplete()) { - handleBoundedCompletion(); - return; - } - logDebugReport(); } catch (Exception e) { @@ -1995,6 +1987,10 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) activelyReadingTaskGroups.clear(); partitionGroups.clear(); partitionOffsets.clear(); + if (ioConfig.isBounded()) { + initializeBoundedPartitionGroups(); + stateManager.maybeSetState(SupervisorStateManager.BasicState.RUNNING); + } } else { if (!checkSourceMetadataMatch(dataSourceMetadata)) { throw new IAE( @@ -2098,9 +2094,9 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) /** * Reset offsets with the data source metadata. If checkpoints exist, the resulting stored offsets will be a union of * existing checkpointed offsets and provided offsets; any checkpointed offsets not specified in the metadata will be - * preserved as-is. If checkpoints don't exist, the provided reset datasource metdadata will be inserted into + * preserved as-is. If checkpoints don't exist, the provided reset datasource metadata will be inserted into * the metadata storage. Once the offsets are reset, any active tasks serving the partition offsets will be restarted. - * @param dataSourceMetadata Required reset data source metdata. Assumed that the metadata is validated. + * @param dataSourceMetadata Required reset data source metadata. Assumed that the metadata is validated. */ public void resetOffsetsInternal(@Nonnull final DataSourceMetadata dataSourceMetadata) { diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java index 437ea6b92922..16b02e05f1fb 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManager.java @@ -123,18 +123,14 @@ public SupervisorStateManager(SupervisorStateManagerConfig supervisorStateManage /** * Certain states are only valid if the supervisor hasn't had a successful iteration. This method checks if there's * been at least one successful iteration, and if applicable, sets supervisor state to an appropriate new state. - * STOPPING and COMPLETED are terminal states that cannot transition to any other state. + * A STOPPING supervisor cannot transition to any other state as this state is final. * This method must be thread-safe as multiple threads trying to update may lead to an invalid state. */ public synchronized void maybeSetState(State proposedState) { - // Terminal states (STOPPING, COMPLETED) take precedence over all other states - if (BasicState.STOPPING.equals(this.supervisorState) || BasicState.COMPLETED.equals(this.supervisorState)) { - // Already in a terminal state, cannot transition - return; - } else if (BasicState.STOPPING.equals(proposedState) || BasicState.COMPLETED.equals(proposedState)) { - // Transitioning to a terminal state - supervisorState = proposedState; + if (BasicState.STOPPING.equals(this.supervisorState) || BasicState.STOPPING.equals(proposedState)) { + // STOPPING takes precedence over all other states + supervisorState = BasicState.STOPPING; return; } @@ -204,11 +200,11 @@ public void markRunFinished() consecutiveSuccessfulRuns = currentRunSuccessful ? consecutiveSuccessfulRuns + 1 : 0; consecutiveFailedRuns = currentRunSuccessful ? 0 : consecutiveFailedRuns + 1; - // If the supervisor is not IDLE, try to set the state to RUNNING or SUSPENDED. + // If the supervisor is not IDLE or COMPLETED, try to set the state to RUNNING or SUSPENDED. // This will be rejected if we haven't had atLeastOneSuccessfulRun // (in favor of the more specific states for the initial run) and will instead trigger setting the state to an // unhealthy one if we are now over the error thresholds. - if (!isIdle()) { + if (!isIdle() && !isCompleted()) { maybeSetState(healthySteadyState); } // reset for next run @@ -245,6 +241,11 @@ public boolean isIdle() return SupervisorStateManager.BasicState.IDLE.equals(supervisorState); } + public boolean isCompleted() + { + return SupervisorStateManager.BasicState.COMPLETED.equals(supervisorState); + } + protected Deque getRecentEventsQueue() { return recentEventsQueue; From c9181f025713915977dc7bdc237069d160189791 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 29 Apr 2026 19:36:38 -0700 Subject: [PATCH 11/52] Handle edge case where startOffset equals endOffset --- .../supervisor/SeekableStreamSupervisor.java | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) 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 30d40a5a1438..a8c6be0aa22d 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 @@ -4484,8 +4484,39 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { private boolean hasTaskGroupReachedBoundedEnd(int groupId) { BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); + Map startOffsets = + convertBoundedConfigMap(boundedConfig.getStartSequenceNumbers()); Map endOffsets = convertBoundedConfigMap(boundedConfig.getEndSequenceNumbers()); + + Set partitionsInGroup = partitionGroups.get(groupId); + if (partitionsInGroup == null || partitionsInGroup.isEmpty()) { + return false; + } + + // Check if start >= end for all partitions (empty range) + // If so, there's no work to do - treat as already complete + boolean allPartitionsEmptyRange = true; + for (PartitionIdType partition : partitionsInGroup) { + SequenceOffsetType start = startOffsets.get(partition); + SequenceOffsetType end = endOffsets.get(partition); + if (!isOffsetAtOrBeyond(start, end)) { + allPartitionsEmptyRange = false; + break; + } + } + + if (allPartitionsEmptyRange) { + log.warn( + "TaskGroup[%d] has empty range for all partitions (start >= end). " + + "No work to do, marking as complete. Start: %s, End: %s", + groupId, + startOffsets, + endOffsets + ); + return true; + } + Map currentOffsets = getOffsetsFromMetadataStorage(); log.info( @@ -4500,11 +4531,6 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) return false; // No progress yet, task hasn't completed } - Set partitionsInGroup = partitionGroups.get(groupId); - if (partitionsInGroup == null || partitionsInGroup.isEmpty()) { - return false; - } - // Check if ALL partitions in this group have reached their end offsets for (PartitionIdType partition : partitionsInGroup) { SequenceOffsetType endOffset = endOffsets.get(partition); From 9e85331b8b5701b17a72385054d62f0702df0f51 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 29 Apr 2026 19:54:41 -0700 Subject: [PATCH 12/52] Compare Kinesis sequence numbers using BigInteger --- .../kinesis/supervisor/KinesisSupervisor.java | 22 ++++++++++++++++--- .../supervisor/KinesisSupervisorTest.java | 1 - 2 files changed, 19 insertions(+), 4 deletions(-) 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 d62872e01d51..7e0cdc02628c 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 @@ -53,6 +53,7 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -60,6 +61,7 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.math.BigInteger; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -384,9 +386,23 @@ protected boolean isShardExpirationMarker(String seqNum) @Override protected boolean isOffsetAtOrBeyond(String current, String target) { - // Kinesis sequence numbers are comparable strings - // They can be compared lexicographically to determine order - return current.compareTo(target) >= 0; + // Kinesis sequence numbers are decimal numeric strings that must be compared numerically. + // Use BigInteger because Kinesis sequence numbers can be very large (128-bit). + try { + BigInteger currentNum = new BigInteger(current); + BigInteger targetNum = new BigInteger(target); + return currentNum.compareTo(targetNum) >= 0; + } + catch (NumberFormatException e) { + throw new IAE( + StringUtils.format( + "Invalid Kinesis sequence number. Expected numeric string but got current=[%s], target=[%s]", + current, + target + ), + e + ); + } } @Override diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 0dfba54fa07c..3c5b958521fe 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -4812,7 +4812,6 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() offset = supervisor.createSequenceOffsetFromObject(100); Assert.assertEquals("100", offset); - // Test offset comparison (lexicographic) Assert.assertTrue(supervisor.isOffsetAtOrBeyond( "49590338271512257353759162668991891722121171891717232706", "49590338271490256608559692538361571095921575989136588898" From 9a32ce09d59b5330e4fc7267b0e0f5764acc7fa3 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 29 Apr 2026 20:38:58 -0700 Subject: [PATCH 13/52] Remove stale test case --- .../SupervisorStateManagerTest.java | 27 ------------------- 1 file changed, 27 deletions(-) diff --git a/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java b/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java index 043992998f72..544086ef842c 100644 --- a/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java +++ b/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java @@ -67,33 +67,6 @@ public void testIdleConfigSerde() Assert.assertEquals(60000, stateManagerConfig.getInactiveAfterMillis()); } - @Test - public void testCompletedStateIsTerminal() - { - stateManagerConfig = new SupervisorStateManagerConfig(); - SupervisorStateManager supervisorStateManager = new SupervisorStateManager( - stateManagerConfig, - false - ); - - // Start in PENDING state - Assert.assertEquals(SupervisorStateManager.BasicState.PENDING, supervisorStateManager.getSupervisorState()); - - // Transition to COMPLETED - supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.COMPLETED); - Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); - - // Attempt to transition out of COMPLETED should be ignored - supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.RUNNING); - Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); - - supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.IDLE); - Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); - - supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.PENDING); - Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); - } - @Test public void testStoppingStateIsTerminal() { From b04e907e97ee38718a7e1726e63a8da83d4998a2 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 1 May 2026 10:04:18 -0700 Subject: [PATCH 14/52] Remove redundant validation of boundedStreamConfig --- .../SeekableStreamSupervisorSpec.java | 36 ----- .../SeekableStreamSupervisorSpecTest.java | 145 ------------------ 2 files changed, 181 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index bee2994d7b6c..83763d08a422 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -262,42 +262,6 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept if (!this.getSource().equals(other.getSource())) { throw InvalidInput.exception(ILLEGAL_INPUT_SOURCE_UPDATE_ERROR_MESSAGE, this.getSource(), other.getSource()); } - - // Validate bounded stream configuration - validateBoundedStreamConfig(other); - } - - /** - * Validates bounded stream configuration for the supervisor spec. - * - * @param spec the supervisor spec to validate - * @throws DruidException if the bounded stream configuration is invalid - */ - protected void validateBoundedStreamConfig(SeekableStreamSupervisorSpec spec) throws DruidException - { - SeekableStreamSupervisorIOConfig ioConfig = spec.getIoConfig(); - - if (ioConfig.isBounded()) { - // Validate partition consistency - BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); - if (!boundedConfig.getStartSequenceNumbers().keySet().equals(boundedConfig.getEndSequenceNumbers().keySet())) { - throw InvalidInput.exception( - "Bounded stream config has mismatched partitions. Start: %s, End: %s", - boundedConfig.getStartSequenceNumbers().keySet(), - boundedConfig.getEndSequenceNumbers().keySet() - ); - } - - // Warn if useConcurrentLocks is not enabled - Map context = spec.getContext(); - if (context == null || !Boolean.TRUE.equals(context.get("useConcurrentLocks"))) { - log.warn( - "Bounded stream processing without 'useConcurrentLocks=true' may fail " + - "if other supervisors are running or segments already exist for these intervals. " + - "Consider setting useConcurrentLocks=true in the supervisor context." - ); - } - } } @Override 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 b914da4ebb17..0efad16bc276 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 @@ -1171,151 +1171,6 @@ public String getSource() originalSpec.validateSpecUpdateTo(proposedSpecSameSource); } - @Test - public void test_validateBoundedStreamConfig_WithValidConfig() - { - mockIngestionSchema(); - - Map startOffsets = ImmutableMap.of("0", 100, "1", 200); - Map endOffsets = ImmutableMap.of("0", 500, "1", 600); - BoundedStreamConfig validConfig = new BoundedStreamConfig(startOffsets, endOffsets); - - SeekableStreamSupervisorIOConfig ioConfigWithValidBounded = new SeekableStreamSupervisorIOConfig( - "stream", - new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), false, false, false), - 1, - 1, - new Period("PT1H"), - new Period("P1D"), - new Period("PT30S"), - false, - new Period("PT30M"), - null, - null, - null, - LagAggregator.DEFAULT, - null, - null, - null, - null, - validConfig - ) - { - }; - - SeekableStreamSupervisorIngestionSpec ingestionWithValidBounded = EasyMock.mock(SeekableStreamSupervisorIngestionSpec.class); - EasyMock.expect(ingestionWithValidBounded.getIOConfig()).andReturn(ioConfigWithValidBounded).anyTimes(); - EasyMock.expect(ingestionWithValidBounded.getDataSchema()).andReturn(dataSchema).anyTimes(); - EasyMock.expect(ingestionWithValidBounded.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); - EasyMock.replay(ingestionWithValidBounded); - - TestSeekableStreamSupervisorSpec spec = new TestSeekableStreamSupervisorSpec( - ingestionSchema, - ImmutableMap.of("key", "value"), - false, - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - indexTaskClientFactory, - mapper, - emitter, - monitorSchedulerConfig, - rowIngestionMetersFactory, - supervisorStateManagerConfig, - supervisor4, - "id1" - ); - - TestSeekableStreamSupervisorSpec specWithValidBounded = new TestSeekableStreamSupervisorSpec( - ingestionWithValidBounded, - ImmutableMap.of("key", "value"), - false, - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - indexTaskClientFactory, - mapper, - emitter, - monitorSchedulerConfig, - rowIngestionMetersFactory, - supervisorStateManagerConfig, - supervisor4, - "id1" - ); - - // Should not throw - spec.validateBoundedStreamConfig(specWithValidBounded); - } - - @Test - @SuppressWarnings({"rawtypes", "unchecked"}) - public void test_validateBoundedStreamConfig_WithMismatchedPartitions() - { - mockIngestionSchema(); - - // Create a mock BoundedStreamConfig that returns mismatched partition sets - BoundedStreamConfig mismatchedConfig = EasyMock.mock(BoundedStreamConfig.class); - Map startMap = ImmutableMap.of("0", 100L, "1", 200L); - Map endMap = ImmutableMap.of("0", 500L, "2", 600L); - EasyMock.expect(mismatchedConfig.getStartSequenceNumbers()).andStubReturn(startMap); - EasyMock.expect(mismatchedConfig.getEndSequenceNumbers()).andStubReturn(endMap); - EasyMock.replay(mismatchedConfig); - - SeekableStreamSupervisorIOConfig ioConfigWithMismatchedBounded = EasyMock.mock(SeekableStreamSupervisorIOConfig.class); - EasyMock.expect(ioConfigWithMismatchedBounded.isBounded()).andReturn(true).anyTimes(); - EasyMock.expect(ioConfigWithMismatchedBounded.getBoundedStreamConfig()).andReturn(mismatchedConfig).anyTimes(); - EasyMock.replay(ioConfigWithMismatchedBounded); - - SeekableStreamSupervisorIngestionSpec ingestionWithMismatchedBounded = EasyMock.mock(SeekableStreamSupervisorIngestionSpec.class); - EasyMock.expect(ingestionWithMismatchedBounded.getIOConfig()).andReturn(ioConfigWithMismatchedBounded).anyTimes(); - EasyMock.expect(ingestionWithMismatchedBounded.getDataSchema()).andReturn(dataSchema).anyTimes(); - EasyMock.expect(ingestionWithMismatchedBounded.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); - EasyMock.replay(ingestionWithMismatchedBounded); - - TestSeekableStreamSupervisorSpec spec = new TestSeekableStreamSupervisorSpec( - ingestionSchema, - ImmutableMap.of("key", "value"), - false, - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - indexTaskClientFactory, - mapper, - emitter, - monitorSchedulerConfig, - rowIngestionMetersFactory, - supervisorStateManagerConfig, - supervisor4, - "id1" - ); - - TestSeekableStreamSupervisorSpec specWithMismatchedBounded = new TestSeekableStreamSupervisorSpec( - ingestionWithMismatchedBounded, - ImmutableMap.of("key", "value"), - false, - taskStorage, - taskMaster, - indexerMetadataStorageCoordinator, - indexTaskClientFactory, - mapper, - emitter, - monitorSchedulerConfig, - rowIngestionMetersFactory, - supervisorStateManagerConfig, - supervisor4, - "id1" - ); - - MatcherAssert.assertThat( - assertThrows(DruidException.class, () -> spec.validateBoundedStreamConfig(specWithMismatchedBounded)), - new DruidExceptionMatcher( - DruidException.Persona.USER, - DruidException.Category.INVALID_INPUT, - "invalidInput" - ).expectMessageContains("Bounded stream config has mismatched partitions") - ); - } - @Test public void test_dynamicAllocationNotice_skipsScalingAndEmitsReason_ifTasksArePublishing() throws InterruptedException { From 8e6dfb817094e58780481b2d81da579c9df71446 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 1 May 2026 10:28:38 -0700 Subject: [PATCH 15/52] Throw DruidException with ADMIN persona for BoundedStreamConfig --- .../supervisor/BoundedStreamConfig.java | 32 +++++++++------- .../supervisor/BoundedStreamConfigTest.java | 38 +++++++++++++------ 2 files changed, 45 insertions(+), 25 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java index 6d8caf0686d3..14c06deaa369 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfig.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; +import org.apache.druid.error.DruidException; import java.util.Map; @@ -50,21 +50,25 @@ public BoundedStreamConfig( @JsonProperty("endSequenceNumbers") Map endSequenceNumbers ) { - this.startSequenceNumbers = Preconditions.checkNotNull(startSequenceNumbers, "startSequenceNumbers"); - this.endSequenceNumbers = Preconditions.checkNotNull(endSequenceNumbers, "endSequenceNumbers"); + if (startSequenceNumbers == null || startSequenceNumbers.isEmpty() || + endSequenceNumbers == null || endSequenceNumbers.isEmpty()) { + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("startSequenceNumbers and endSequenceNumbers cannot be null or empty"); + } - // Validation - Preconditions.checkArgument( - !startSequenceNumbers.isEmpty(), - "startSequenceNumbers cannot be empty" - ); + if (!startSequenceNumbers.keySet().equals(endSequenceNumbers.keySet())) { + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "startSequenceNumbers and endSequenceNumbers must have matching partition sets. Start: %s, End: %s", + startSequenceNumbers.keySet(), + endSequenceNumbers.keySet() + ); + } - Preconditions.checkArgument( - startSequenceNumbers.keySet().equals(endSequenceNumbers.keySet()), - "startSequenceNumbers and endSequenceNumbers must have matching partition sets. Start: %s, End: %s", - startSequenceNumbers.keySet(), - endSequenceNumbers.keySet() - ); + this.startSequenceNumbers = startSequenceNumbers; + this.endSequenceNumbers = endSequenceNumbers; } @JsonProperty diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java index 9c86b4d9a6d7..d527eca6b1df 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; import org.junit.Assert; import org.junit.Test; @@ -53,12 +54,12 @@ public void testConstructorWithNullStartSequenceNumbers() Map endOffsets = new HashMap<>(); endOffsets.put("0", 500L); - NullPointerException ex = Assert.assertThrows( - NullPointerException.class, + DruidException ex = Assert.assertThrows( + DruidException.class, () -> new BoundedStreamConfig(null, endOffsets) ); - Assert.assertTrue(ex.getMessage().contains("startSequenceNumbers")); + Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); } @Test @@ -67,12 +68,12 @@ public void testConstructorWithNullEndSequenceNumbers() Map startOffsets = new HashMap<>(); startOffsets.put("0", 100L); - NullPointerException ex = Assert.assertThrows( - NullPointerException.class, + DruidException ex = Assert.assertThrows( + DruidException.class, () -> new BoundedStreamConfig(startOffsets, null) ); - Assert.assertTrue(ex.getMessage().contains("endSequenceNumbers")); + Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); } @Test @@ -82,12 +83,27 @@ public void testConstructorWithEmptyStartSequenceNumbers() Map endOffsets = new HashMap<>(); endOffsets.put("0", 500L); - IllegalArgumentException ex = Assert.assertThrows( - IllegalArgumentException.class, + DruidException ex = Assert.assertThrows( + DruidException.class, () -> new BoundedStreamConfig(startOffsets, endOffsets) ); - Assert.assertTrue(ex.getMessage().contains("startSequenceNumbers cannot be empty")); + Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); + } + + @Test + public void testConstructorWithEmptyEndSequenceNumbers() + { + Map startOffsets = new HashMap<>(); + startOffsets.put("0", 100L); + Map endOffsets = new HashMap<>(); + + DruidException ex = Assert.assertThrows( + DruidException.class, + () -> new BoundedStreamConfig(startOffsets, endOffsets) + ); + + Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); } @Test @@ -98,8 +114,8 @@ public void testConstructorWithMismatchedPartitions() Map endOffsets = new HashMap<>(); endOffsets.put("1", 500L); - IllegalArgumentException ex = Assert.assertThrows( - IllegalArgumentException.class, + DruidException ex = Assert.assertThrows( + DruidException.class, () -> new BoundedStreamConfig(startOffsets, endOffsets) ); From f03abb6c5dee2445339597433c84e9d7ae1165d4 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 1 May 2026 12:22:40 -0700 Subject: [PATCH 16/52] Clean up unused Logger --- .../seekablestream/supervisor/SeekableStreamSupervisorSpec.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index 83763d08a422..3cf21ebb4aa1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -37,7 +37,6 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.NoopTaskAutoScaler; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -50,7 +49,6 @@ public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec { - private static final Logger log = new Logger(SeekableStreamSupervisorSpec.class); protected static final String ILLEGAL_INPUT_SOURCE_UPDATE_ERROR_MESSAGE = "Update of the input source stream from [%s] to [%s] is not supported for a running supervisor." From ae9083f5157eae331c7dcb76828ecda1181574af Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 1 May 2026 12:28:55 -0700 Subject: [PATCH 17/52] javadoc and comment cleanup for isBoundedWorkComplete --- .../supervisor/SeekableStreamSupervisor.java | 19 ------------------- 1 file changed, 19 deletions(-) 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 a8c6be0aa22d..efa9ac850f74 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 @@ -4610,20 +4610,10 @@ private Map getEndOffsetsForGroup(int group } /** - * Check if all bounded tasks have completed. - * Called after createNewTasks() in runInternal to ensure tasks have been created first. - * * For bounded supervisors, we determine completion by checking if new tasks would be created. * In createNewTasks(), bounded mode checks hasTaskGroupReachedBoundedEnd() before creating tasks. * If that returns true (offsets reached), no new tasks are created. - * * So completion is: no active tasks, no pending tasks, and createNewTasks() chose not to create any. - * This is indicated by empty task groups after createNewTasks() has run. - * - * We do NOT separately check metadata storage here because: - * 1. Metadata may contain stale offsets from previous supervisor runs - * 2. createNewTasks() already does the offset checking logic - * 3. If tasks were killed/failed and work is incomplete, createNewTasks() will recreate them * * @return true if all bounded work is complete, false otherwise */ @@ -4640,13 +4630,6 @@ private boolean isBoundedWorkComplete() if (!noActiveTasks || !noPendingTasks) { return false; } - - // At this point, no tasks are running. Since createNewTasks() already ran, - // if tasks aren't running it means either: - // A) Tasks completed successfully and offset targets were reached (don't recreate) - // B) Tasks failed/killed and haven't reached targets (will recreate next run) - // - // To distinguish, we check if createNewTasks() would create new tasks. // If hasTaskGroupReachedBoundedEnd() returns false for any group, createNewTasks() // will create tasks next iteration, so we're not complete. for (Integer groupId : partitionGroups.keySet()) { @@ -4656,8 +4639,6 @@ private boolean isBoundedWorkComplete() } } - // All groups have reached their end offsets and no tasks are running. - // Work is complete! log.info("All bounded tasks completed for supervisor[%s]", supervisorId); return true; } From f8a313ba9211f03e08621d787caa46c16f1425e5 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 1 May 2026 14:23:44 -0700 Subject: [PATCH 18/52] Add embedded test for bounded ingestion --- .../indexing/KafkaBoundedSupervisorTest.java | 176 ++++++++++++++++++ .../kafka/simulate/KafkaResource.java | 38 ++++ .../supervisor/KafkaIOConfigBuilder.java | 10 +- 3 files changed, 223 insertions(+), 1 deletion(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java new file mode 100644 index 000000000000..29f4d40f888a --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -0,0 +1,176 @@ +/* + * 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.testing.embedded.indexing; + +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; +import org.apache.druid.testing.embedded.StreamIngestResource; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +/** + * Tests for bounded Kafka supervisors (one-time ingestion with explicit start/end offsets). + */ +public class KafkaBoundedSupervisorTest extends StreamIndexTestBase +{ + private final KafkaResource kafkaServer = new KafkaResource(); + + @Override + protected StreamIngestResource getStreamIngestResource() + { + return kafkaServer; + } + + @Test + public void test_boundedSupervisor_ingestsDataAndCompletes() + { + final String topic = IdUtils.getRandomId(); + kafkaServer.createTopicWithPartitions(topic, 2); + + // Publish records before creating supervisor + final int totalRecords = publish1kRecords(topic, false); + + // Get the current end offsets for all partitions + Map endOffsets = kafkaServer.getPartitionOffsets(topic); + Assertions.assertEquals(2, endOffsets.size(), "Should have 2 partitions"); + + // Create bounded config with start offset 0 and current end offsets + Map startOffsets = new HashMap<>(); + startOffsets.put("0", 0L); + startOffsets.put("1", 0L); + + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + // Create bounded supervisor + final KafkaSupervisorSpec supervisor = createBoundedKafkaSupervisor( + kafkaServer, + topic, + boundedConfig + ); + + cluster.callApi().postSupervisor(supervisor); + + // Wait for records to be ingested + waitUntilPublishedRecordsAreIngested(totalRecords); + + // Wait for supervisor to transition to COMPLETED state + waitForSupervisorToComplete(supervisor.getId()); + + // Verify row count + verifyRowCount(totalRecords); + + // Verify supervisor is in COMPLETED state + final SupervisorStatus status = cluster.callApi().getSupervisorStatus(supervisor.getId()); + Assertions.assertEquals("COMPLETED", status.getState()); + Assertions.assertTrue(status.isHealthy()); + } + + @Test + public void test_boundedSupervisor_withEmptyRange_completesImmediately() + { + final String topic = IdUtils.getRandomId(); + kafkaServer.createTopicWithPartitions(topic, 1); + + // Publish some records + publish1kRecords(topic, false); + + // Get current offset + Map currentOffsets = kafkaServer.getPartitionOffsets(topic); + Long currentOffset = currentOffsets.get("0"); + + // Create bounded config with start == end (empty range) + Map offsets = new HashMap<>(); + offsets.put("0", currentOffset); + + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(offsets, offsets); + + // Create bounded supervisor + final KafkaSupervisorSpec supervisor = createBoundedKafkaSupervisor( + kafkaServer, + topic, + boundedConfig + ); + + cluster.callApi().postSupervisor(supervisor); + + // Wait for supervisor to transition to COMPLETED state + waitForSupervisorToComplete(supervisor.getId()); + + // Verify supervisor is in COMPLETED state + final SupervisorStatus status = cluster.callApi().getSupervisorStatus(supervisor.getId()); + Assertions.assertEquals("COMPLETED", status.getState()); + } + + private KafkaSupervisorSpec createBoundedKafkaSupervisor( + KafkaResource kafkaServer, + String topic, + BoundedStreamConfig boundedConfig + ) + { + return createKafkaSupervisor(kafkaServer) + .withIoConfig(io -> io + .withKafkaInputFormat(new JsonInputFormat(null, null, null, null, null)) + .withBoundedStreamConfig(boundedConfig) + ) + .build(dataSource, topic); + } + + private void waitForSupervisorToComplete(String supervisorId) + { + // Wait for supervisor to reach COMPLETED state + int maxAttempts = 60; // 60 seconds timeout + int attempt = 0; + + while (attempt < maxAttempts) { + try { + SupervisorStatus status = cluster.callApi().getSupervisorStatus(supervisorId); + if ("COMPLETED".equals(status.getState())) { + return; + } + Thread.sleep(1000); + attempt++; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for supervisor to complete", e); + } + catch (Exception e) { + // Supervisor might not be found immediately, retry + attempt++; + try { + Thread.sleep(1000); + } + catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting", ie); + } + } + } + + Assertions.fail("Supervisor did not complete within timeout"); + } +} diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java index 2df7c6a7be10..245454d4b2bc 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/KafkaResource.java @@ -27,9 +27,12 @@ import org.apache.kafka.clients.admin.CreatePartitionsResult; import org.apache.kafka.clients.admin.NewPartitions; import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.testcontainers.kafka.KafkaContainer; @@ -242,6 +245,41 @@ public Admin newAdminClient() return Admin.create(commonClientProperties()); } + /** + * Returns the current end offsets for all partitions in the specified topic. + * The returned map has partition IDs as String keys and end offsets as Long values. + */ + public Map getPartitionOffsets(String topicName) + { + Map offsets = new HashMap<>(); + + // Add required deserializer config + Map props = new HashMap<>(consumerProperties()); + props.put("key.deserializer", ByteArrayDeserializer.class.getName()); + props.put("value.deserializer", ByteArrayDeserializer.class.getName()); + + try (KafkaConsumer consumer = new KafkaConsumer<>(props)) { + // Get all partitions for the topic + List partitions = new ArrayList<>(); + consumer.partitionsFor(topicName).forEach( + partitionInfo -> partitions.add(new TopicPartition(topicName, partitionInfo.partition())) + ); + + // Get end offsets for all partitions + Map endOffsets = consumer.endOffsets(partitions); + + // Convert to String keys + for (Map.Entry entry : endOffsets.entrySet()) { + offsets.put(String.valueOf(entry.getKey().partition()), entry.getValue()); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + + return offsets; + } + @Override public String toString() { 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..ac8a9e1d2f7b 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 @@ -21,6 +21,7 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.kafkainput.KafkaInputFormat; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.indexing.seekablestream.supervisor.SupervisorIOConfigBuilder; import java.util.Map; @@ -33,6 +34,7 @@ public class KafkaIOConfigBuilder extends SupervisorIOConfigBuilder consumerProperties; + private BoundedStreamConfig boundedStreamConfig; public KafkaIOConfigBuilder withTopic(String topic) { @@ -68,6 +70,12 @@ public KafkaIOConfigBuilder withKafkaInputFormat(InputFormat valueFormat) return this; } + public KafkaIOConfigBuilder withBoundedStreamConfig(BoundedStreamConfig boundedStreamConfig) + { + this.boundedStreamConfig = boundedStreamConfig; + return this; + } + @Override public KafkaSupervisorIOConfig build() { @@ -94,7 +102,7 @@ public KafkaSupervisorIOConfig build() stopTaskCount, null, null, - null + boundedStreamConfig ); } } From 5965ac4de5f1e344906ddbe1a94afce29b85b04e Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 1 May 2026 20:03:58 -0700 Subject: [PATCH 19/52] Add boundedStreamConfig to SeekableStreamDataSourceMetadata for metadata validation --- .../RabbitStreamDataSourceMetadata.java | 17 +++++- .../RabbitStreamIndexTaskIOConfig.java | 7 ++- .../RabbitStreamIndexTaskRunner.java | 5 +- .../supervisor/RabbitStreamSupervisor.java | 3 +- .../kafka/KafkaDataSourceMetadata.java | 18 +++++- .../kafka/KafkaIndexTaskIOConfig.java | 10 +++- .../indexing/kafka/KafkaIndexTaskRunner.java | 5 +- .../kafka/supervisor/KafkaSupervisor.java | 3 +- .../kinesis/KinesisDataSourceMetadata.java | 18 +++++- .../kinesis/KinesisIndexTaskIOConfig.java | 10 +++- .../kinesis/KinesisIndexTaskRunner.java | 5 +- .../kinesis/supervisor/KinesisSupervisor.java | 6 +- .../SeekableStreamDataSourceMetadata.java | 26 ++++++++- .../SeekableStreamIndexTaskIOConfig.java | 14 ++++- .../supervisor/BoundedStreamConfig.java | 22 +++++++ .../supervisor/SeekableStreamSupervisor.java | 57 +++++++++++++++++-- .../TestSeekableStreamIndexTaskIOConfig.java | 3 +- .../SeekableStreamSupervisorStateTest.java | 6 +- .../SeekableStreamSupervisorTestBase.java | 3 +- .../org/apache/druid/cli/CliPeonTest.java | 3 +- 20 files changed, 204 insertions(+), 37 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.java index 5cafa1d7925d..ba8c79747c12 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.java @@ -25,15 +25,26 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; + +import javax.annotation.Nullable; public class RabbitStreamDataSourceMetadata extends SeekableStreamDataSourceMetadata { @JsonCreator public RabbitStreamDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamSequenceNumbers partitions) + @JsonProperty("partitions") SeekableStreamSequenceNumbers partitions, + @JsonProperty("boundedStreamConfig") @Nullable BoundedStreamConfig boundedStreamConfig) + { + super(partitions, boundedStreamConfig); + } + + // Backward compatibility constructor + public RabbitStreamDataSourceMetadata( + SeekableStreamSequenceNumbers partitions) { - super(partitions); + this(partitions, null); } @Override @@ -52,6 +63,6 @@ public DataSourceMetadata asStartMetadata() protected SeekableStreamDataSourceMetadata createConcreteDataSourceMetaData( SeekableStreamSequenceNumbers seekableStreamSequenceNumbers) { - return new RabbitStreamDataSourceMetadata(seekableStreamSequenceNumbers); + return new RabbitStreamDataSourceMetadata(seekableStreamSequenceNumbers, getBoundedStreamConfig()); } } 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..580578007a34 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 @@ -27,6 +27,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -54,7 +55,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("boundedStreamConfig") @Nullable BoundedStreamConfig boundedStreamConfig ) { super( @@ -66,7 +68,8 @@ public RabbitStreamIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + boundedStreamConfig ); 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/RabbitStreamIndexTaskRunner.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskRunner.java index e34f0d9e6d15..101afd4570dd 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskRunner.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskRunner.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.java.util.emitter.EmittingLogger; import javax.annotation.Nonnull; @@ -93,7 +94,9 @@ protected SeekableStreamEndSequenceNumbers deserializePartitionsFr protected SeekableStreamDataSourceMetadata createDataSourceMetadata( SeekableStreamSequenceNumbers partitions) { - return new RabbitStreamDataSourceMetadata(partitions); + // Include bounded config if this is a bounded task + BoundedStreamConfig boundedConfig = task.getIOConfig().getBoundedStreamConfig(); + return new RabbitStreamDataSourceMetadata(partitions, boundedConfig); } @Override 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 d3907f16008d..23327d6fd376 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(), + rabbitConfig.getBoundedStreamConfig() // Pass through bounded config ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java index cfa308099489..04a6c2ba9285 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadata.java @@ -27,11 +27,13 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.utils.CollectionUtils; import org.apache.kafka.common.TopicPartition; +import javax.annotation.Nullable; import java.util.Comparator; import java.util.Map; @@ -41,7 +43,8 @@ public class KafkaDataSourceMetadata extends SeekableStreamDataSourceMetadata kafkaPartitions + @JsonProperty("partitions") SeekableStreamSequenceNumbers kafkaPartitions, + @JsonProperty("boundedStreamConfig") @Nullable BoundedStreamConfig boundedStreamConfig ) { super(kafkaPartitions == null @@ -60,7 +63,16 @@ public KafkaDataSourceMetadata( ((SeekableStreamEndSequenceNumbers) kafkaPartitions).getTopic(), kafkaPartitions.getPartitionSequenceNumberMap(), ((SeekableStreamEndSequenceNumbers) kafkaPartitions).getPartitionOffsetMap() - )); + ), + boundedStreamConfig); + } + + // Backward compatibility constructor + public KafkaDataSourceMetadata( + SeekableStreamSequenceNumbers kafkaPartitions + ) + { + this(kafkaPartitions, null); } @Override @@ -81,7 +93,7 @@ protected SeekableStreamDataSourceMetadata createConc SeekableStreamSequenceNumbers seekableStreamSequenceNumbers ) { - return new KafkaDataSourceMetadata(seekableStreamSequenceNumbers); + return new KafkaDataSourceMetadata(seekableStreamSequenceNumbers, getBoundedStreamConfig()); } @Override 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..1d36d2996cce 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 @@ -29,6 +29,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.extension.KafkaConfigOverrides; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -64,7 +65,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("boundedStreamConfig") @Nullable BoundedStreamConfig boundedStreamConfig ) { super( @@ -78,7 +80,8 @@ public KafkaIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + boundedStreamConfig ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); @@ -128,7 +131,8 @@ public KafkaIndexTaskIOConfig( inputFormat, configOverrides, KafkaSupervisorIOConfig.DEFAULT_IS_MULTI_TOPIC, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + null ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java index 3da19165771b..ac4ab0e7a1c5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskRunner.java @@ -34,6 +34,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.utils.CollectionUtils; @@ -174,7 +175,9 @@ protected SeekableStreamDataSourceMetadata createData SeekableStreamSequenceNumbers partitions ) { - return new KafkaDataSourceMetadata(partitions); + // Include bounded config if this is a bounded task + BoundedStreamConfig boundedConfig = task.getIOConfig().getBoundedStreamConfig(); + return new KafkaDataSourceMetadata(partitions, boundedConfig); } @Override 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 d85e56fc4918..247e8f1deb9e 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 @@ -227,7 +227,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ioConfig.getInputFormat(), kafkaIoConfig.getConfigOverrides(), kafkaIoConfig.isMultiTopic(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + kafkaIoConfig.getBoundedStreamConfig() // Pass through bounded config ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java index d3c1630cf909..5cd9b2943fbe 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisDataSourceMetadata.java @@ -25,15 +25,27 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; + +import javax.annotation.Nullable; public class KinesisDataSourceMetadata extends SeekableStreamDataSourceMetadata { @JsonCreator public KinesisDataSourceMetadata( - @JsonProperty("partitions") SeekableStreamSequenceNumbers kinesisPartitions + @JsonProperty("partitions") SeekableStreamSequenceNumbers kinesisPartitions, + @JsonProperty("boundedStreamConfig") @Nullable BoundedStreamConfig boundedStreamConfig + ) + { + super(kinesisPartitions, boundedStreamConfig); + } + + // Backward compatibility constructor + public KinesisDataSourceMetadata( + SeekableStreamSequenceNumbers kinesisPartitions ) { - super(kinesisPartitions); + this(kinesisPartitions, null); } @Override @@ -54,6 +66,6 @@ protected KinesisDataSourceMetadata createConcreteDataSourceMetaData( SeekableStreamSequenceNumbers seekableStreamSequenceNumbers ) { - return new KinesisDataSourceMetadata(seekableStreamSequenceNumbers); + return new KinesisDataSourceMetadata(seekableStreamSequenceNumbers, getBoundedStreamConfig()); } } 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..d5b29f2e41ed 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 @@ -27,6 +27,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -79,7 +80,8 @@ public KinesisIndexTaskIOConfig( @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, @JsonProperty("awsAssumedRoleArn") String awsAssumedRoleArn, @JsonProperty("awsExternalId") String awsExternalId, - @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes + @JsonProperty("refreshRejectionPeriodsInMinutes") Long refreshRejectionPeriodsInMinutes, + @JsonProperty("boundedStreamConfig") @Nullable BoundedStreamConfig boundedStreamConfig ) { super( @@ -91,7 +93,8 @@ public KinesisIndexTaskIOConfig( minimumMessageTime, maximumMessageTime, inputFormat, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + boundedStreamConfig ); Preconditions.checkArgument( getEndSequenceNumbers().getPartitionSequenceNumberMap() @@ -139,7 +142,8 @@ public KinesisIndexTaskIOConfig( fetchDelayMillis, awsAssumedRoleArn, awsExternalId, - refreshRejectionPeriodsInMinutes + refreshRejectionPeriodsInMinutes, + null // boundedStreamConfig ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 3e2f656329f9..8c3bf5c57fa9 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -97,7 +98,9 @@ protected SeekableStreamDataSourceMetadata createDataSourceMetad SeekableStreamSequenceNumbers partitions ) { - return new KinesisDataSourceMetadata(partitions); + // Include bounded config if this is a bounded task + BoundedStreamConfig boundedConfig = task.getIOConfig().getBoundedStreamConfig(); + return new KinesisDataSourceMetadata(partitions, boundedConfig); } @Override 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 7e0cdc02628c..416bec1acbd3 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 @@ -135,6 +135,9 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( return new KinesisIndexTaskIOConfig( groupId, baseSequenceName, + null, + null, + null, new SeekableStreamStartSequenceNumbers<>( ioConfig.getStream(), startPartitions, @@ -149,7 +152,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ioConfig.getFetchDelayMillis(), ioConfig.getAwsAssumedRoleArn(), ioConfig.getAwsExternalId(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + ioConfig.getBoundedStreamConfig() // Pass through bounded config ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java index 4ea1e992da0d..0b42da98c9ec 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamDataSourceMetadata.java @@ -21,8 +21,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.java.util.common.IAE; +import javax.annotation.Nullable; import java.util.Objects; public abstract class SeekableStreamDataSourceMetadata @@ -30,11 +32,23 @@ public abstract class SeekableStreamDataSourceMetadata seekableStreamSequenceNumbers; + @Nullable + private final BoundedStreamConfig boundedStreamConfig; + public SeekableStreamDataSourceMetadata( SeekableStreamSequenceNumbers seekableStreamSequenceNumbers ) + { + this(seekableStreamSequenceNumbers, null); + } + + public SeekableStreamDataSourceMetadata( + SeekableStreamSequenceNumbers seekableStreamSequenceNumbers, + @Nullable BoundedStreamConfig boundedStreamConfig + ) { this.seekableStreamSequenceNumbers = seekableStreamSequenceNumbers; + this.boundedStreamConfig = boundedStreamConfig; } @JsonProperty("partitions") @@ -43,6 +57,13 @@ public SeekableStreamSequenceNumbers getSee return seekableStreamSequenceNumbers; } + @Nullable + @JsonProperty("boundedStreamConfig") + public BoundedStreamConfig getBoundedStreamConfig() + { + return boundedStreamConfig; + } + @Override public boolean isValidStart() { @@ -105,13 +126,14 @@ public boolean equals(Object o) return false; } SeekableStreamDataSourceMetadata that = (SeekableStreamDataSourceMetadata) o; - return Objects.equals(getSeekableStreamSequenceNumbers(), that.getSeekableStreamSequenceNumbers()); + return Objects.equals(getSeekableStreamSequenceNumbers(), that.getSeekableStreamSequenceNumbers()) && + Objects.equals(boundedStreamConfig, that.boundedStreamConfig); } @Override public int hashCode() { - return seekableStreamSequenceNumbers.hashCode(); + return Objects.hash(seekableStreamSequenceNumbers, boundedStreamConfig); } @Override 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..cf79c7ab4818 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 @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.InputFormat; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.segment.indexing.IOConfig; import org.joda.time.DateTime; @@ -42,6 +43,7 @@ public abstract class SeekableStreamIndexTaskIOConfig taskGroup.tasks.size()) { + // Count both actively reading tasks and pending completion tasks for this group + int totalTasksForGroup = taskGroup.tasks.size(); + CopyOnWriteArrayList pendingCompletionGroups = pendingCompletionTaskGroups.get(groupId); + if (pendingCompletionGroups != null) { + for (TaskGroup pendingGroup : pendingCompletionGroups) { + totalTasksForGroup += pendingGroup.tasks.size(); + } + } + + if (ioConfig.getReplicas() > totalTasksForGroup) { log.info( "Number of tasks[%d] does not match configured numReplicas[%d] in taskGroup[%d], creating more tasks.", - taskGroup.tasks.size(), ioConfig.getReplicas(), groupId + totalTasksForGroup, ioConfig.getReplicas(), groupId ); - createTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size()); + createTasksForGroup(groupId, ioConfig.getReplicas() - totalTasksForGroup); createdTask = true; } } @@ -4346,9 +4355,11 @@ private Map> generate ) { // Existing logic for both streaming and bounded mode - // Bounded mode will fall back to bounded start offsets in getOffsetFromStorageForPartition() ImmutableMap.Builder> builder = ImmutableMap.builder(); + final DataSourceMetadata dataSourceMetadata = retrieveDataSourceMetadata(); final Map metadataOffsets = getOffsetsFromMetadataStorage(); + final BoundedStreamConfig metadataBoundedConfig = getBoundedConfigFromMetadata(dataSourceMetadata); + for (PartitionIdType partitionId : partitionGroups.get(groupId)) { SequenceOffsetType sequence = partitionOffsets.get(partitionId); @@ -4362,7 +4373,8 @@ private Map> generate // get the sequence from metadata storage (if available) or Kafka/Kinesis (otherwise) OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition( partitionId, - metadataOffsets + metadataOffsets, + metadataBoundedConfig ); if (offsetFromStorage != null) { @@ -4373,6 +4385,18 @@ private Map> generate return builder.build(); } + /** + * Extracts BoundedStreamConfig from DataSourceMetadata if available. + */ + @Nullable + private BoundedStreamConfig getBoundedConfigFromMetadata(@Nullable DataSourceMetadata metadata) + { + if (metadata instanceof SeekableStreamDataSourceMetadata) { + return ((SeekableStreamDataSourceMetadata) metadata).getBoundedStreamConfig(); + } + return null; + } + /** * Queries the dataSource metadata table to see if there is a previous ending sequence for this partition. If it * doesn't find any data, it will retrieve the latest or earliest Kafka/Kinesis sequence depending on the @@ -4380,11 +4404,32 @@ private Map> generate */ private OrderedSequenceNumber getOffsetFromStorageForPartition( PartitionIdType partition, - final Map metadataOffsets + final Map metadataOffsets, + @Nullable final BoundedStreamConfig metadataBoundedConfig ) { SequenceOffsetType sequence = metadataOffsets.get(partition); if (sequence != null) { + // In bounded mode, check if the metadata's bounded config matches the current supervisor's config + if (ioConfig.isBounded()) { + BoundedStreamConfig currentBoundedConfig = ioConfig.getBoundedStreamConfig(); + + // If configs don't match (or metadata has no config), throw exception + if (!currentBoundedConfig.equals(metadataBoundedConfig)) { + throw DruidException.forPersona(DruidException.Persona.ADMIN) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Bounded supervisor detected existing metadata from a different run. " + + "Metadata bounded config [%s] does not match current config [%s]. " + + "To start a new bounded ingestion, either: " + + "(1) use the supervisor reset API to clear existing metadata, or " + + "(2) use a different supervisor ID / datasource.", + metadataBoundedConfig, + currentBoundedConfig + ); + } + } + log.debug("Getting sequence [%s] from metadata storage for partition [%s]", sequence, partition); if (!taskTuningConfig.isSkipSequenceNumberAvailabilityCheck()) { if (!checkOffsetAvailability(partition, sequence)) { 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..4f4fe8afc01e 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(), + 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 a17d59759e76..357b7aa99ebb 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 @@ -3146,7 +3146,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( minimumMessageTime, maximumMessageTime, ioConfig.getInputFormat(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + null ) { }; @@ -3529,7 +3530,8 @@ private static SeekableStreamIndexTaskIOConfig createTaskIoConfigExt( minimumMessageTime, maximumMessageTime, ioConfig.getInputFormat(), - ioConfig.getTaskDuration().getStandardMinutes() + ioConfig.getTaskDuration().getStandardMinutes(), + 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 dad426a755c5..9f8625d09cb9 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(), + null ) { }; 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 8851c16a4522..f7f6eb7f60ab 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(), + null ); } } From 9e6694857b3b70e78968ce78893ed37537f6c966 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Fri, 1 May 2026 20:24:56 -0700 Subject: [PATCH 20/52] Revert pendingCompletionGroups check --- .../supervisor/SeekableStreamSupervisor.java | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) 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 3f96c0d5ed5d..f48731140b68 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 @@ -4303,21 +4303,12 @@ private void createNewTasks() throws JsonProcessingException continue; } - // Count both actively reading tasks and pending completion tasks for this group - int totalTasksForGroup = taskGroup.tasks.size(); - CopyOnWriteArrayList pendingCompletionGroups = pendingCompletionTaskGroups.get(groupId); - if (pendingCompletionGroups != null) { - for (TaskGroup pendingGroup : pendingCompletionGroups) { - totalTasksForGroup += pendingGroup.tasks.size(); - } - } - - if (ioConfig.getReplicas() > totalTasksForGroup) { + if (ioConfig.getReplicas() > taskGroup.tasks.size()) { log.info( "Number of tasks[%d] does not match configured numReplicas[%d] in taskGroup[%d], creating more tasks.", - totalTasksForGroup, ioConfig.getReplicas(), groupId + taskGroup.tasks.size(), ioConfig.getReplicas(), groupId ); - createTasksForGroup(groupId, ioConfig.getReplicas() - totalTasksForGroup); + createTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size()); createdTask = true; } } From 902e1185fc1242968bf69f315fddcf6ebe3342e9 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Sat, 2 May 2026 16:14:33 -0700 Subject: [PATCH 21/52] Unit test fix --- .../druid/indexing/kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../indexing/kinesis/supervisor/KinesisSupervisorTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 1f3bedae78fb..91ffea040391 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 @@ -3671,7 +3671,7 @@ public void testGetOffsetFromStorageForPartitionWithResetOffsetAutomatically() t new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, singlePartitionMap(topic, 1, -100L, 2, 200L)) ) - ).times(3); + ).times(4); // getOffsetFromStorageForPartition() throws an exception when the offsets are automatically reset. // Since getOffsetFromStorageForPartition() is called per partition, all partitions can't be reset at the same time. // Instead, subsequent partitions will be reset in the following supervisor runs. diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 3c5b958521fe..57fd30b608ff 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -2832,7 +2832,7 @@ public void testGetOffsetFromStorageForPartitionWithResetOffsetAutomatically() t new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "100", SHARD_ID2, "200")) ) ) - .times(2); + .times(3); // Since shard 2 was in metadata before but is not in the list of shards returned by the record supplier, // it gets deleted from metadata (it is an expired shard) From 670749c45ce2bad78d42bb30384c33eb549bf968 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Sat, 2 May 2026 17:12:43 -0700 Subject: [PATCH 22/52] embedded-test for metadata mismatch --- .../indexing/KafkaBoundedSupervisorTest.java | 118 ++++++++++++++++++ 1 file changed, 118 insertions(+) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index 29f4d40f888a..baba5f0d60f5 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -25,6 +25,7 @@ import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; +import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.testing.embedded.StreamIngestResource; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -37,6 +38,7 @@ */ public class KafkaBoundedSupervisorTest extends StreamIndexTestBase { + private static final EmittingLogger log = new EmittingLogger(KafkaBoundedSupervisorTest.class); private final KafkaResource kafkaServer = new KafkaResource(); @Override @@ -139,6 +141,83 @@ private KafkaSupervisorSpec createBoundedKafkaSupervisor( .build(dataSource, topic); } + @Test + public void test_boundedSupervisor_withMismatchedMetadata_is_unhealthy() + { + final String topic = IdUtils.getRandomId(); + kafkaServer.createTopicWithPartitions(topic, 2); + + // Publish records before creating supervisor + final int totalRecords = publish1kRecords(topic, false); + + // Get the current end offsets for all partitions + Map currentOffsets = kafkaServer.getPartitionOffsets(topic); + Assertions.assertEquals(2, currentOffsets.size(), "Should have 2 partitions"); + + // Create first bounded config - ingest only the first 100 records from each partition + Map startOffsets1 = new HashMap<>(); + startOffsets1.put("0", 0L); + startOffsets1.put("1", 0L); + + Map endOffsets1 = new HashMap<>(); + endOffsets1.put("0", 100L); + endOffsets1.put("1", 100L); + + BoundedStreamConfig boundedConfig1 = new BoundedStreamConfig(startOffsets1, endOffsets1); + + // Create first bounded supervisor and run it to completion + final KafkaSupervisorSpec supervisor1 = createBoundedKafkaSupervisor( + kafkaServer, + topic, + boundedConfig1 + ); + + cluster.callApi().postSupervisor(supervisor1); + + // Wait for records to be ingested (approximately 200 records total from both partitions) + waitUntilPublishedRecordsAreIngested(200); + + // Wait for supervisor to transition to COMPLETED state + waitForSupervisorToComplete(supervisor1.getId()); + + // Verify supervisor is in COMPLETED state + final SupervisorStatus status1 = cluster.callApi().getSupervisorStatus(supervisor1.getId()); + Assertions.assertEquals("COMPLETED", status1.getState()); + + // Now try to create a second bounded supervisor with different bounded config on the same datasource + // The key is that the second supervisor's range [50, 200] overlaps with the first supervisor's + // checkpointed offsets (~100), so the metadata mismatch will be detected. If the bounded end is less + // then the checkpointed offset then the Supervisor detects that no work is needed and the Supervisor + // completes silently. + Map startOffsets2 = new HashMap<>(); + startOffsets2.put("0", 50L); // Different start offset + startOffsets2.put("1", 50L); + + Map endOffsets2 = new HashMap<>(); + endOffsets2.put("0", 200L); // Different end offset + endOffsets2.put("1", 200L); + + BoundedStreamConfig boundedConfig2 = new BoundedStreamConfig(startOffsets2, endOffsets2); + + final KafkaSupervisorSpec supervisor2 = createBoundedKafkaSupervisor( + kafkaServer, + topic, + boundedConfig2 + ); + + // Post the second supervisor (it should use the same supervisor ID/datasource) + cluster.callApi().postSupervisor(supervisor2); + + // Wait for the supervisor to process and detect the metadata mismatch + // The exception we're testing for is thrown and logged, and causes the supervisor to become unhealthy + waitForSupervisorToBeUnhealthy(supervisor2.getId()); + + // Verify the supervisor is unhealthy + final SupervisorStatus status2 = cluster.callApi().getSupervisorStatus(supervisor2.getId()); + Assertions.assertFalse(status2.isHealthy(), "Supervisor should be unhealthy after detecting metadata mismatch"); + Assertions.assertEquals("UNHEALTHY_SUPERVISOR", status2.getState(), "Supervisor state should be UNHEALTHY_SUPERVISOR"); + } + private void waitForSupervisorToComplete(String supervisorId) { // Wait for supervisor to reach COMPLETED state @@ -173,4 +252,43 @@ private void waitForSupervisorToComplete(String supervisorId) Assertions.fail("Supervisor did not complete within timeout"); } + + private void waitForSupervisorToBeUnhealthy(String supervisorId) + { + // Wait for supervisor to become unhealthy after detecting the metadata mismatch + int maxAttempts = 30; // 30 seconds timeout + int attempt = 0; + + while (attempt < maxAttempts) { + try { + SupervisorStatus status = cluster.callApi().getSupervisorStatus(supervisorId); + + // The supervisor should become unhealthy when the exception is thrown + if (!status.isHealthy()) { + log.info("Supervisor became unhealthy with state: %s", status.getDetailedState()); + return; + } + + Thread.sleep(1000); + attempt++; + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for supervisor to become unhealthy", e); + } + catch (Exception e) { + // Supervisor might not be found immediately, retry + attempt++; + try { + Thread.sleep(1000); + } + catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting", ie); + } + } + } + + Assertions.fail("Supervisor did not become unhealthy due to metadata mismatch within timeout"); + } } From 2457caf710578b63fc242e187927028decc68278 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Sat, 2 May 2026 17:54:27 -0700 Subject: [PATCH 23/52] Remove unused var --- .../testing/embedded/indexing/KafkaBoundedSupervisorTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index baba5f0d60f5..c99c55b565b9 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -147,9 +147,6 @@ public void test_boundedSupervisor_withMismatchedMetadata_is_unhealthy() final String topic = IdUtils.getRandomId(); kafkaServer.createTopicWithPartitions(topic, 2); - // Publish records before creating supervisor - final int totalRecords = publish1kRecords(topic, false); - // Get the current end offsets for all partitions Map currentOffsets = kafkaServer.getPartitionOffsets(topic); Assertions.assertEquals(2, currentOffsets.size(), "Should have 2 partitions"); From 3943ad0d61bdc6ba57056b518c71111f99263023 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Sun, 3 May 2026 14:29:00 -0700 Subject: [PATCH 24/52] Unit test fix --- .../testing/embedded/indexing/KafkaBoundedSupervisorTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index c99c55b565b9..43e74afd221c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -146,6 +146,7 @@ public void test_boundedSupervisor_withMismatchedMetadata_is_unhealthy() { final String topic = IdUtils.getRandomId(); kafkaServer.createTopicWithPartitions(topic, 2); + publish1kRecords(topic, false); // Get the current end offsets for all partitions Map currentOffsets = kafkaServer.getPartitionOffsets(topic); From 395fa9acefef164c97d841d961a75d52332b19ae Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 09:42:41 -0700 Subject: [PATCH 25/52] Add boundedStreamConfig documentation --- docs/ingestion/supervisor.md | 52 ++++++++++++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/docs/ingestion/supervisor.md b/docs/ingestion/supervisor.md index 592f0328cbe9..98b33b77e050 100644 --- a/docs/ingestion/supervisor.md +++ b/docs/ingestion/supervisor.md @@ -65,6 +65,7 @@ For configuration properties specific to Kafka and Kinesis, see [Kafka I/O confi |`lateMessageRejectionPeriod`|ISO 8601 period|Configures tasks to reject messages with timestamps earlier than this period before the task was created. For example, if this property is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a streaming and a nightly batch ingestion pipeline. You can specify only one of the late message rejection properties.|No|| |`earlyMessageRejectionPeriod`|ISO 8601 period|Configures tasks to reject messages with timestamps later than this period after the task reached its task duration. For example, if this property is set to `PT1H`, the task duration is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps later than `2016-01-01T14:00Z`. Tasks sometimes run past their task duration, such as in cases of supervisor failover.|No|| |`stopTaskCount`|Integer|Limits the number of ingestion tasks Druid can cycle at any given time. If not set, Druid can cycle all tasks at the same time. If set to a value less than `taskCount`, your cluster needs fewer available slots to run the supervisor. You can save costs by scaling down your ingestion tier, but this can lead to slower cycle times and lag. See [`stopTaskCount`](#stoptaskcount) for more information.|No|`taskCount` value| +|`boundedStreamConfig`|Object|Configures the supervisor for bounded (one-time) ingestion with explicit start and end offsets. When set, the supervisor creates tasks that read from `startSequenceNumbers` to `endSequenceNumbers`, then automatically terminates when all data is ingested. The bounded configuration is stored with datasource metadata; if a supervisor is restarted or a new supervisor is created with different offsets for the same datasource, it will fail. To retry with different offsets, use the supervisor reset API to clear metadata or use a different datasource. Useful for backfills and historical reprocessing. See [Bounded stream configuration](#bounded-stream-configuration) for details.|No|null| |`serverPriorityToReplicas`|Object (`Map`)|Map of server priorities to the number of replicas per priority. When set, each task replica is assigned a server priority that corresponds to `druid.server.priority` on the Peon process to enable query isolation for mixed workloads using [query routing strategies](../configuration/index.md#query-routing). If not configured, the `replicas` setting applies and all task replicas are assigned a default priority of 0.

For example, setting `serverPriorityToReplicas` to `{"1": 2, "0": 1}` creates 2 task replicas with `druid.server.priority=1` and 1 task replica with `druid.server.priority=0` per task group. This configuration scales proportionally with `taskCount`. For example, if `taskCount` is set to 5, this results in 15 total tasks - 10 tasks with priority 1 and 5 tasks with priority 0. If both `replicas` and `serverPriorityToReplicas` are set, the sum of replicas in `serverPriorityToReplicas` must equal `replicas`.|No|null| #### Task autoscaler @@ -245,6 +246,57 @@ Before you set `stopTaskCount`, note the following: - The [task autoscaler](#task-autoscaler) ignores `stopTaskCount` when shutting down tasks in response to a task count change. The task autoscaler needs to redistribute partitions across tasks, which requires all tasks to be shut down. - If you set `stopTaskCount` to a value less than `taskCount`, Druid cycles the longest running tasks first, then other tasks up to the value set. +#### Bounded stream configuration + +Use `boundedStreamConfig` to configure one-time ingestion from a specific range of offsets. This is useful for backfilling historical data or reprocessing data with different configurations. + +The `boundedStreamConfig` object contains the following properties: + +|Property|Type|Description|Required| +|--------|----|-----------|--------| +|`startSequenceNumbers`|Object|Map of partition IDs to start offsets (inclusive for Kafka, inclusive for Kinesis).|Yes| +|`endSequenceNumbers`|Object|Map of partition IDs to end offsets (exclusive for Kafka, inclusive for Kinesis).|Yes| + +When configured, the supervisor: +1. Creates tasks that start reading from `startSequenceNumbers` +2. Tasks automatically stop when they reach `endSequenceNumbers` +3. Supervisor does not create replacement tasks after tasks complete +4. Supervisor transitions to `COMPLETED` state and terminates when all tasks finish + +**Metadata consistency:** The bounded configuration is stored in datasource metadata along with checkpointed offsets. If you restart the supervisor or create a new supervisor with a different `boundedStreamConfig` for the same datasource, the supervisor will fail with an error. To start a new bounded ingestion with different offsets, either: +- Use the [supervisor reset API](../api-reference/supervisor-api.md#reset-a-supervisor) to clear existing metadata +- Use a different datasource name + +**Example (Kafka):** + +```json +{ + "type": "kafka", + "spec": { + "ioConfig": { + "topic": "my-topic", + "inputFormat": { + "type": "json" + }, + "boundedStreamConfig": { + "startSequenceNumbers": { + "0": 1000, + "1": 2000, + "2": 1500 + }, + "endSequenceNumbers": { + "0": 5000, + "1": 6000, + "2": 5500 + } + } + } + } +} +``` + +This configuration ingests data from partition 0 offsets 1000-4999, partition 1 offsets 2000-5999, and partition 2 offsets 1500-5499. + ### Tuning configuration The `tuningConfig` object is optional. If you don't specify the `tuningConfig` object, Druid uses the default configuration settings. From 4cde39f4684f771fb8d3536214d6e5b39f88ae54 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 11:03:22 -0700 Subject: [PATCH 26/52] Fix spellcheck --- website/.spelling | 3 +++ 1 file changed, 3 insertions(+) diff --git a/website/.spelling b/website/.spelling index 99e767c48910..3872960c997d 100644 --- a/website/.spelling +++ b/website/.spelling @@ -291,6 +291,7 @@ averagers backend backfills backfilled +backfilling backpressure base64 baseTime @@ -303,6 +304,7 @@ boolean breakpoint broadcasted bucketSize +checkpointed checksums classpath clickstream @@ -508,6 +510,7 @@ prepends prepopulated preprocessing priori +reprocessing procs processFromRaw programmatically From 7e86ec6b5bb31b2019bd32b4e4784676fd4acd98 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 12:45:13 -0700 Subject: [PATCH 27/52] Increase code coverage --- .../kafka/supervisor/KafkaSupervisorTest.java | 235 ++++++++++++++++++ 1 file changed, 235 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 91ffea040391..b9c775de1a7f 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 @@ -6333,4 +6333,239 @@ public boolean isTaskCurrent(int taskGroupId, String taskId, Map t return isTaskCurrentReturn; } } + + @Test + public void testBoundedStreamConfig_tasksIncludeBoundedConfig() throws Exception + { + Map startOffsets = ImmutableMap.of("0", 10L, "1", 20L, "2", 30L); + Map endOffsets = ImmutableMap.of("0", 100L, "1", 100L, "2", 100L); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + supervisor = getTestableSupervisorWithBoundedConfig(1, 1, "PT1H", boundedConfig); + addSomeEvents(100); + + Capture captured = Capture.newInstance(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata(null) + ).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + // Task should be created with bounded config + KafkaIndexTask task = captured.getValue(); + Assert.assertNotNull(task); + Assert.assertEquals(boundedConfig, task.getIOConfig().getBoundedStreamConfig()); + + // Start offsets should come from bounded config + KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); + Assert.assertEquals(10L, (long) taskConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 0))); + Assert.assertEquals(20L, (long) taskConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 1))); + Assert.assertEquals(30L, (long) taskConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 2))); + + // End offsets should match bounded config + Assert.assertEquals(100L, (long) taskConfig.getEndSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 0))); + } + + @Test + public void testBoundedStreamConfig_withCheckpoint_resumesFromCheckpoint() throws Exception + { + Map startOffsets = ImmutableMap.of("0", 0L, "1", 0L, "2", 0L); + Map endOffsets = ImmutableMap.of("0", 100L, "1", 100L, "2", 100L); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + supervisor = getTestableSupervisorWithBoundedConfig(1, 1, "PT1H", boundedConfig); + addSomeEvents(100); + + // Simulate existing checkpoint from previous run with same bounded config + SeekableStreamStartSequenceNumbers checkpointSequences = + new SeekableStreamStartSequenceNumbers<>( + topic, + ImmutableMap.of( + new KafkaTopicPartition(false, topic, 0), 50L, + new KafkaTopicPartition(false, topic, 1), 60L, + new KafkaTopicPartition(false, topic, 2), 70L + ), + ImmutableSet.of() + ); + + Capture captured = Capture.newInstance(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata(checkpointSequences, boundedConfig) + ).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + // Task should be created with bounded config + KafkaIndexTask task = captured.getValue(); + Assert.assertNotNull(task); + Assert.assertEquals(boundedConfig, task.getIOConfig().getBoundedStreamConfig()); + + // Start offsets should resume from checkpoint + KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); + Assert.assertEquals(50L, (long) taskConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 0))); + Assert.assertEquals(60L, (long) taskConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 1))); + Assert.assertEquals(70L, (long) taskConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 2))); + + // End offsets should still match bounded config + Assert.assertEquals(100L, (long) taskConfig.getEndSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 0))); + } + + @Test + public void testBoundedStreamConfig_endOffsetsSetCorrectly() throws Exception + { + Map startOffsets = ImmutableMap.of("0", 0L, "1", 0L, "2", 0L); + Map endOffsets = ImmutableMap.of("0", 150L, "1", 250L, "2", 350L); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + supervisor = getTestableSupervisorWithBoundedConfig(1, 1, "PT1H", boundedConfig); + addSomeEvents(350); + + Capture captured = Capture.newInstance(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata(null) + ).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + verifyAll(); + + // Task end offsets should match bounded config (not Long.MAX_VALUE) + KafkaIndexTask task = captured.getValue(); + KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); + + Assert.assertEquals(150L, (long) taskConfig.getEndSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 0))); + Assert.assertEquals(250L, (long) taskConfig.getEndSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 1))); + Assert.assertEquals(350L, (long) taskConfig.getEndSequenceNumbers() + .getPartitionSequenceNumberMap() + .get(new KafkaTopicPartition(false, topic, 2))); + } + + private TestableKafkaSupervisor getTestableSupervisorWithBoundedConfig( + int replicas, + int taskCount, + String duration, + BoundedStreamConfig boundedConfig + ) + { + final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); + consumerProperties.put("myCustomKey", "myCustomValue"); + consumerProperties.put("bootstrap.servers", kafkaHost); + KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( + topic, + null, + INPUT_FORMAT, + replicas, + taskCount, + new Period(duration), + consumerProperties, + null, + null, + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + new Period("P1D"), + new Period("PT30S"), + true, + new Period("PT30M"), + null, + null, + null, + null, + null, + null, + true, + null, + boundedConfig + ); + + KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( + null, + null + ) + { + @Override + public SeekableStreamIndexTaskClient build( + String dataSource, + TaskInfoProvider taskInfoProvider, + SeekableStreamSupervisorTuningConfig tuningConfig, + ScheduledExecutorService connectExec + ) + { + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout()); + Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries()); + return taskClient; + } + }; + + final KafkaSupervisorTuningConfig tuningConfig = tuningConfigBuilder() + .withMaxSavedParseExceptions(10) + .build(); + + return new TestableKafkaSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new KafkaSupervisorSpec( + null, + null, + dataSchema, + tuningConfig, + kafkaSupervisorIOConfig, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + supervisorConfig + ), + rowIngestionMetersFactory + ); + } } From d985dc4a23e98d771ed7e97a65b26d31a6a5770a Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 13:35:24 -0700 Subject: [PATCH 28/52] Increase coverage for BoundedStreamConfig --- .../supervisor/BoundedStreamConfigTest.java | 143 ++++++++++++++++++ 1 file changed, 143 insertions(+) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java index d527eca6b1df..f67c1312586c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java @@ -196,4 +196,147 @@ public void testDeserializationWithMixedTypes() throws Exception Assert.assertEquals(2, config.getStartSequenceNumbers().size()); Assert.assertEquals(2, config.getEndSequenceNumbers().size()); } + + @Test + public void testEquals_sameObject() + { + Map start = new HashMap<>(); + start.put("0", 0L); + Map end = new HashMap<>(); + end.put("0", 100L); + + BoundedStreamConfig config = new BoundedStreamConfig(start, end); + + Assert.assertEquals(config, config); + } + + @Test + public void testEquals_equalObjects() + { + Map start1 = new HashMap<>(); + start1.put("0", 0L); + Map end1 = new HashMap<>(); + end1.put("0", 100L); + + Map start2 = new HashMap<>(); + start2.put("0", 0L); + Map end2 = new HashMap<>(); + end2.put("0", 100L); + + BoundedStreamConfig config1 = new BoundedStreamConfig(start1, end1); + BoundedStreamConfig config2 = new BoundedStreamConfig(start2, end2); + + Assert.assertEquals(config1, config2); + Assert.assertEquals(config1.hashCode(), config2.hashCode()); + } + + @Test + public void testEquals_nullObject() + { + Map start = new HashMap<>(); + start.put("0", 0L); + Map end = new HashMap<>(); + end.put("0", 100L); + + BoundedStreamConfig config = new BoundedStreamConfig(start, end); + + Assert.assertNotEquals(config, null); + } + + @Test + public void testEquals_differentClass() + { + Map start = new HashMap<>(); + start.put("0", 0L); + Map end = new HashMap<>(); + end.put("0", 100L); + + BoundedStreamConfig config = new BoundedStreamConfig(start, end); + + Assert.assertNotEquals(config, "not a BoundedStreamConfig"); + } + + @Test + public void testEquals_differentStartOffsets() + { + Map start1 = new HashMap<>(); + start1.put("0", 0L); + Map start2 = new HashMap<>(); + start2.put("0", 10L); + Map end = new HashMap<>(); + end.put("0", 100L); + + BoundedStreamConfig config1 = new BoundedStreamConfig(start1, end); + BoundedStreamConfig config2 = new BoundedStreamConfig(start2, end); + + Assert.assertNotEquals(config1, config2); + } + + @Test + public void testEquals_differentEndOffsets() + { + Map start = new HashMap<>(); + start.put("0", 0L); + Map end1 = new HashMap<>(); + end1.put("0", 100L); + Map end2 = new HashMap<>(); + end2.put("0", 200L); + + BoundedStreamConfig config1 = new BoundedStreamConfig(start, end1); + BoundedStreamConfig config2 = new BoundedStreamConfig(start, end2); + + Assert.assertNotEquals(config1, config2); + } + + @Test + public void testHashCode_consistency() + { + Map start = new HashMap<>(); + start.put("0", 0L); + Map end = new HashMap<>(); + end.put("0", 100L); + + BoundedStreamConfig config = new BoundedStreamConfig(start, end); + + int hashCode1 = config.hashCode(); + int hashCode2 = config.hashCode(); + + Assert.assertEquals(hashCode1, hashCode2); + } + + @Test + public void testHashCode_equalObjectsSameHashCode() + { + Map start1 = new HashMap<>(); + start1.put("0", 0L); + Map end1 = new HashMap<>(); + end1.put("0", 100L); + + Map start2 = new HashMap<>(); + start2.put("0", 0L); + Map end2 = new HashMap<>(); + end2.put("0", 100L); + + BoundedStreamConfig config1 = new BoundedStreamConfig(start1, end1); + BoundedStreamConfig config2 = new BoundedStreamConfig(start2, end2); + + Assert.assertEquals(config1.hashCode(), config2.hashCode()); + } + + @Test + public void testToString() + { + Map start = new HashMap<>(); + start.put("0", 0L); + Map end = new HashMap<>(); + end.put("0", 100L); + + BoundedStreamConfig config = new BoundedStreamConfig(start, end); + String str = config.toString(); + + Assert.assertNotNull(str); + Assert.assertTrue(str.contains("BoundedStreamConfig")); + Assert.assertTrue(str.contains("startSequenceNumbers")); + Assert.assertTrue(str.contains("endSequenceNumbers")); + } } From 021e72189831d5f194c25cee856ad44ddda4386a Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 14:11:43 -0700 Subject: [PATCH 29/52] Remove unnecessary test --- .../supervisor/BoundedStreamConfigTest.java | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java index f67c1312586c..8eea77e58ad6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java @@ -197,19 +197,6 @@ public void testDeserializationWithMixedTypes() throws Exception Assert.assertEquals(2, config.getEndSequenceNumbers().size()); } - @Test - public void testEquals_sameObject() - { - Map start = new HashMap<>(); - start.put("0", 0L); - Map end = new HashMap<>(); - end.put("0", 100L); - - BoundedStreamConfig config = new BoundedStreamConfig(start, end); - - Assert.assertEquals(config, config); - } - @Test public void testEquals_equalObjects() { From d23d9c47589b250998971e79138ac9339d5e121c Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 15:27:47 -0700 Subject: [PATCH 30/52] Simplify completion check in createNewTasks --- .../supervisor/SeekableStreamSupervisor.java | 27 +++++-------------- 1 file changed, 7 insertions(+), 20 deletions(-) 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 f48731140b68..ac4a77e813cc 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 @@ -4175,26 +4175,13 @@ private void createNewTasks() throws JsonProcessingException for (Integer groupId : partitionGroups.keySet()) { if (!activelyReadingTaskGroups.containsKey(groupId)) { - // In bounded mode, distinguish between completion and failure - if (ioConfig.isBounded()) { - if (hasTaskGroupReachedBoundedEnd(groupId)) { - // Task group completed successfully - don't recreate - log.debug( - "Bounded taskGroup[%d] has reached end offsets, skipping recreation", - groupId - ); - continue; // Skip creating new task group - } else { - // Task group hasn't reached end - task must have failed, recreate it - log.info( - "Bounded taskGroup[%d] has not reached end offsets (current: %s, target: %s). " + - "Task may have failed, recreating to continue processing.", - groupId, - getCurrentOffsetsForGroup(groupId), - getEndOffsetsForGroup(groupId) - ); - // Fall through to create new task group - } + // In bounded mode, check if task group has completed before recreating + if (ioConfig.isBounded() && hasTaskGroupReachedBoundedEnd(groupId)) { + log.debug( + "Bounded taskGroup[%d] has reached end offsets, skipping recreation", + groupId + ); + continue; // Skip creating new task group } log.info("Creating new taskGroup[%d] for partitions[%s].", groupId, partitionGroups.get(groupId)); From 42ada89a6030cd9573f89d0a198f1e2fa264b9d5 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 17:35:10 -0700 Subject: [PATCH 31/52] Remove unused function --- .../supervisor/SeekableStreamSupervisor.java | 23 ------------------- 1 file changed, 23 deletions(-) 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 ac4a77e813cc..e328791d3906 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 @@ -4587,29 +4587,6 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) return true; // All partitions have reached their end offsets } - /** - * Get current offsets for all partitions in a task group from metadata storage. - */ - private Map getCurrentOffsetsForGroup(int groupId) - { - Map allOffsets = getOffsetsFromMetadataStorage(); - if (allOffsets == null || allOffsets.isEmpty()) { - return Collections.emptyMap(); - } - - Set partitionsInGroup = partitionGroups.get(groupId); - if (partitionsInGroup == null) { - return Collections.emptyMap(); - } - - return partitionsInGroup.stream() - .filter(allOffsets::containsKey) - .collect(Collectors.toMap( - p -> p, - allOffsets::get - )); - } - /** * Get end offsets for all partitions in a task group from bounded config. */ From ed589c21142412c1aa9ba6963866ca19932e209e Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 17:50:11 -0700 Subject: [PATCH 32/52] Unit test bounded supervisor completion --- .../kafka/supervisor/KafkaSupervisorTest.java | 35 +++++++++++++++++++ 1 file changed, 35 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 b9c775de1a7f..cc99c54ff439 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 @@ -6482,6 +6482,41 @@ public void testBoundedStreamConfig_endOffsetsSetCorrectly() throws Exception .get(new KafkaTopicPartition(false, topic, 2))); } + @Test + public void testBoundedStreamConfig_allPartitionsEmptyRange_completesImmediately() throws Exception + { + // All partitions have start == end (nothing to process) + Map startOffsets = ImmutableMap.of("0", 100L, "1", 100L, "2", 100L); + Map endOffsets = ImmutableMap.of("0", 100L, "1", 100L, "2", 100L); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + supervisor = getTestableSupervisorWithBoundedConfig(1, 1, "PT1H", boundedConfig); + addSomeEvents(100); + + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata(null) + ).anyTimes(); + // registerListener may or may not be called depending on when completion is detected + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + EasyMock.expectLastCall().anyTimes(); + replayAll(); + + supervisor.start(); + supervisor.runInternal(); + + // With all partitions having empty ranges, supervisor should detect completion + // State should transition to COMPLETED + SupervisorReport report = supervisor.getStatus(); + Assert.assertEquals( + SupervisorStateManager.BasicState.COMPLETED, + report.getPayload().getDetailedState() + ); + } + private TestableKafkaSupervisor getTestableSupervisorWithBoundedConfig( int replicas, int taskCount, From 234bc82d3b25c479005e1d361ce6ad88ae0c6bc5 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 19:02:48 -0700 Subject: [PATCH 33/52] Improve coverage on RabbitStreamSupervisor --- .../RabbitStreamSupervisorTest.java | 49 ++++++++++++------- 1 file changed, 31 insertions(+), 18 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index 19a5cb86d298..5ff274df357e 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -67,6 +67,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; public class RabbitStreamSupervisorTest extends EasyMockSupport { @@ -467,13 +468,13 @@ public void test_doesTaskMatchSupervisor() } @Test - public void testBoundedModeConfiguration() + public void testBoundedModeCreateTasksWithCorrectOffsets() { - ImmutableMap startOffsets = ImmutableMap.of( + Map startOffsets = ImmutableMap.of( "queue-0", 100, "queue-1", 200 ); - ImmutableMap endOffsets = ImmutableMap.of( + Map endOffsets = ImmutableMap.of( "queue-0", 500, "queue-1", 600 ); @@ -484,7 +485,7 @@ public void testBoundedModeConfiguration() INPUT_FORMAT, 1, 1, - new Period("PT1H"), + new Period("PT30S"), null, null, null, @@ -501,32 +502,44 @@ public void testBoundedModeConfiguration() ); Assert.assertTrue(rabbitSupervisorIOConfig.isBounded()); - Assert.assertNotNull(rabbitSupervisorIOConfig.getBoundedStreamConfig()); - Assert.assertEquals(2, rabbitSupervisorIOConfig.getBoundedStreamConfig().getStartSequenceNumbers().size()); - Assert.assertEquals(2, rabbitSupervisorIOConfig.getBoundedStreamConfig().getEndSequenceNumbers().size()); - // Create supervisor to test type conversion methods - supervisor = getSupervisor( - "supervisorId", - 1, - 1, - false, - "PT1H", + final RabbitStreamIndexTaskClientFactory taskClientFactory = new RabbitStreamIndexTaskClientFactory(null, OBJECT_MAPPER); + final RabbitStreamSupervisorSpec spec = new RabbitStreamSupervisorSpec( null, null, dataSchema, - tuningConfig + tuningConfig, + rabbitSupervisorIOConfig, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + new SupervisorStateManagerConfig() + ); + + supervisor = new RabbitStreamSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + spec, + rowIngestionMetersFactory ); - // Test createPartitionIdFromString + // Test type conversion methods String queueName = supervisor.createPartitionIdFromString("queue-0"); Assert.assertEquals("queue-0", queueName); - // Test createSequenceOffsetFromObject with Integer Long offset = supervisor.createSequenceOffsetFromObject(100); Assert.assertEquals(Long.valueOf(100L), offset); - // Test createSequenceOffsetFromObject with String offset = supervisor.createSequenceOffsetFromObject("200"); Assert.assertEquals(Long.valueOf(200L), offset); From 1cd928d60e6524228a6a5ded69562472f650c32e Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 20:05:05 -0700 Subject: [PATCH 34/52] Unit test coverage --- .../RabbitStreamIndexTaskIOConfigTest.java | 28 +++++++++++++++++++ .../supervisor/KinesisSupervisorTest.java | 13 +++++++++ 2 files changed, 41 insertions(+) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java index 7783e1487776..b7e72e0c3178 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java @@ -21,12 +21,15 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.indexing.IOConfig; import org.junit.Assert; import org.junit.Test; import java.util.Collections; +import java.util.Map; public class RabbitStreamIndexTaskIOConfigTest { @@ -77,4 +80,29 @@ public void testSerdeWithDefaults() throws Exception Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions()); } + @Test + public void testRabbitStreamDataSourceMetadataWithBoundedConfig() + { + Map startOffsets = ImmutableMap.of("q0", 0L); + Map endOffsets = ImmutableMap.of("q0", 100L); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + SeekableStreamStartSequenceNumbers partitions = + new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of("q0", 10L), Collections.emptySet()); + + RabbitStreamDataSourceMetadata metadata = new RabbitStreamDataSourceMetadata(partitions, boundedConfig); + Assert.assertNotNull(metadata.getBoundedStreamConfig()); + Assert.assertEquals(boundedConfig, metadata.getBoundedStreamConfig()); + } + + @Test + public void testRabbitStreamDataSourceMetadataWithoutBoundedConfig() + { + SeekableStreamStartSequenceNumbers partitions = + new SeekableStreamStartSequenceNumbers<>("stream", ImmutableMap.of("q0", 10L), Collections.emptySet()); + + RabbitStreamDataSourceMetadata metadata = new RabbitStreamDataSourceMetadata(partitions); + Assert.assertNull(metadata.getBoundedStreamConfig()); + } + } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 57fd30b608ff..d7a38b4adb0f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -4826,6 +4826,19 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() )); } + @Test + public void testIsOffsetAtOrBeyond_invalidSequenceNumber() + { + supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); + + Exception e = Assert.assertThrows( + IllegalArgumentException.class, + () -> supervisor.isOffsetAtOrBeyond("not-a-number", "12345") + ); + Assert.assertTrue(e.getMessage().contains("Invalid Kinesis sequence number")); + Assert.assertTrue(e.getMessage().contains("not-a-number")); + } + private List testShardMergePhaseOne() throws Exception { supervisorRecordSupplier.assign(EasyMock.anyObject()); From 094427d360bef87a760c12d47ee452ee0705d04c Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Mon, 4 May 2026 20:49:39 -0700 Subject: [PATCH 35/52] Unit test for IllegalArgumentException for KafkaSupervisor --- .../kafka/supervisor/KafkaSupervisorTest.java | 14 ++++++++++++++ 1 file changed, 14 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 cc99c54ff439..6b91ffefbf91 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 @@ -5570,6 +5570,20 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() throws JsonProcessing Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); } + @Test + public void testCreateSequenceOffsetFromObject_invalidType() + { + Map startOffsets = ImmutableMap.of("0", 0, "1", 0); + Map endOffsets = ImmutableMap.of("0", 100, "1", 100); + supervisor = getTestableSupervisorWithBoundedConfig(1, 1, "PT1H", new BoundedStreamConfig(startOffsets, endOffsets)); + + Exception e = Assert.assertThrows( + IllegalArgumentException.class, + () -> supervisor.createSequenceOffsetFromObject(new Object()) + ); + Assert.assertTrue(e.getMessage().contains("Cannot convert")); + } + private void addSomeEvents(int numEventsPerPartition) throws Exception { // create topic manually From 3ad278d2d405aa5956ddfbd41684ff6c5d83b89c Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 5 May 2026 10:39:42 -0700 Subject: [PATCH 36/52] Check if end offsets are exclusive for bounded work completion --- .../supervisor/RabbitStreamSupervisor.java | 6 ++ .../kafka/supervisor/KafkaSupervisor.java | 6 ++ .../kafka/supervisor/KafkaSupervisorTest.java | 30 +++++++ .../kinesis/supervisor/KinesisSupervisor.java | 6 ++ .../supervisor/KinesisSupervisorTest.java | 89 +++++++++++++++++++ .../supervisor/SeekableStreamSupervisor.java | 26 +++++- .../SeekableStreamSupervisorStateTest.java | 18 ++++ .../SeekableStreamSupervisorTestBase.java | 6 ++ 8 files changed, 183 insertions(+), 4 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 23327d6fd376..6099105b3374 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 @@ -363,6 +363,12 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() return false; } + @Override + protected boolean isEndOffsetExclusive() + { + return true; + } + @Override protected boolean isOffsetAtOrBeyond(Long current, Long target) { 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 247e8f1deb9e..727eb52db272 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 @@ -422,6 +422,12 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() return false; } + @Override + protected boolean isEndOffsetExclusive() + { + return true; + } + @Override public LagStats computeLagStats() { 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 6b91ffefbf91..ab251bdc45fd 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 @@ -6617,4 +6617,34 @@ public SeekableStreamIndexTaskClient build( rowIngestionMetersFactory ); } + + @Test + public void testBoundedMode_equalOffsetsIsEmpty() + { + // Kafka has exclusive end offsets, so start == end represents an EMPTY range + Map startOffsets = ImmutableMap.of("0", 100, "1", 100); + Map endOffsets = ImmutableMap.of("0", 100, "1", 100); + supervisor = getTestableSupervisorWithBoundedConfig(1, 1, "PT1H", new BoundedStreamConfig(startOffsets, endOffsets)); + + // Kafka uses exclusive end offsets + Assert.assertTrue("Kafka should have exclusive end offsets", supervisor.isEndOffsetExclusive()); + + // Verify that start == end is treated correctly based on offset semantics + // For exclusive offsets: start == end means ZERO records (empty) + Long start = 100L; + Long end = 100L; + + // start >= end is true (they're equal) + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(start, end)); + + // For Kafka (exclusive), this IS an empty range + // The empty range check should be: isOffsetAtOrBeyond(start, end) + // Which evaluates to: true (IS empty) + boolean shouldBeEmpty = supervisor.isOffsetAtOrBeyond(start, end); + + Assert.assertTrue( + "For Kafka with exclusive end offsets, start == end should be considered an empty range", + shouldBeEmpty + ); + } } 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 416bec1acbd3..2a9df4052cc9 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 @@ -429,6 +429,12 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() return true; } + @Override + protected boolean isEndOffsetExclusive() + { + return false; + } + // Unlike the Kafka Indexing Service, // Kinesis reports lag metrics measured in time difference in milliseconds between the current sequence number and latest sequence number, // rather than message count. diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index d7a38b4adb0f..9c8ccf2868c2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -4839,6 +4839,95 @@ public void testIsOffsetAtOrBeyond_invalidSequenceNumber() Assert.assertTrue(e.getMessage().contains("not-a-number")); } + @Test + public void testBoundedMode_singleRecordRange_notEmpty() + { + // Kinesis has inclusive end offsets, so start == end represents ONE record, not an empty range + String singleOffset = "49590338271490256608559692538361571095921575989136588898"; + Map startOffsets = ImmutableMap.of(SHARD_ID0, singleOffset); + Map endOffsets = ImmutableMap.of(SHARD_ID0, singleOffset); + + final KinesisSupervisorIOConfig ioConfig = new KinesisSupervisorIOConfig( + STREAM, + INPUT_FORMAT, + "awsEndpoint", + null, + 1, + 1, + new Period("PT1H"), + null, + new Period("PT30M"), + null, + null, + null, + null, + null, + null, + 0, + null, + null, + null, + true, + null, + new BoundedStreamConfig(startOffsets, endOffsets) + ); + + final KinesisIndexTaskClientFactory taskClientFactory = new KinesisIndexTaskClientFactory(null, null); + final KinesisSupervisorSpec spec = new KinesisSupervisorSpec( + null, + null, + dataSchema, + KinesisSupervisorTuningConfig.defaultConfig(), + ioConfig, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + null, + new SupervisorStateManagerConfig() + ); + + supervisor = new TestableKinesisSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + spec, + rowIngestionMetersFactory + ); + + // Kinesis uses inclusive end offsets + Assert.assertFalse("Kinesis should have inclusive end offsets", supervisor.isEndOffsetExclusive()); + + // Verify that start == end is treated correctly based on offset semantics + // For inclusive offsets: start == end means ONE record (not empty) + // For exclusive offsets: start == end means ZERO records (empty) + String start = singleOffset; + String end = singleOffset; + + // start >= end is true (they're equal) + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(start, end)); + + // But for Kinesis (inclusive), this is NOT an empty range + // The empty range check should be: isOffsetAtOrBeyond(start, end) && !start.equals(end) + // Which evaluates to: true && false = false (NOT empty) + boolean isAtOrBeyond = supervisor.isOffsetAtOrBeyond(start, end); + boolean isEqual = start.equals(end); + boolean shouldBeEmpty = isAtOrBeyond && !isEqual; + + Assert.assertFalse( + "For Kinesis with inclusive end offsets, start == end should NOT be considered an empty range", + shouldBeEmpty + ); + } + private List testShardMergePhaseOne() throws Exception { supervisorRecordSupplier.assign(EasyMock.anyObject()); 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 e328791d3906..981b6789af8d 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 @@ -4517,13 +4517,24 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) return false; } - // Check if start >= end for all partitions (empty range) - // If so, there's no work to do - treat as already complete + // Check if all partitions have empty ranges + // For exclusive end offsets (Kafka): start >= end means empty + // For inclusive end offsets (Kinesis): only start > end means empty (start == end is one record) boolean allPartitionsEmptyRange = true; for (PartitionIdType partition : partitionsInGroup) { SequenceOffsetType start = startOffsets.get(partition); SequenceOffsetType end = endOffsets.get(partition); - if (!isOffsetAtOrBeyond(start, end)) { + + boolean isEmpty; + if (isEndOffsetExclusive()) { + // Exclusive: empty if start >= end + isEmpty = isOffsetAtOrBeyond(start, end); + } else { + // Inclusive: empty only if start > end + isEmpty = isOffsetAtOrBeyond(start, end) && !start.equals(end); + } + + if (!isEmpty) { allPartitionsEmptyRange = false; break; } @@ -4531,9 +4542,10 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) if (allPartitionsEmptyRange) { log.warn( - "TaskGroup[%d] has empty range for all partitions (start >= end). " + "TaskGroup[%d] has empty range for all partitions (start %s end). " + "No work to do, marking as complete. Start: %s, End: %s", groupId, + isEndOffsetExclusive() ? ">=" : ">", startOffsets, endOffsets ); @@ -5449,4 +5461,10 @@ protected LagStats aggregatePartitionLags(Map partitionLa * sequences. In Kafka, start offsets are always inclusive. */ protected abstract boolean useExclusiveStartSequenceNumberForNonFirstSequence(); + + /** + * Returns true if end offsets are exclusive (like Kafka), false if inclusive (like Kinesis). + * Used for bounded mode to correctly detect empty ranges: start == end is empty only if exclusive. + */ + protected abstract boolean isEndOffsetExclusive(); } 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 357b7aa99ebb..e781feeb22d3 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 @@ -2615,6 +2615,12 @@ public LagStats computeLagStats() return new LagStats(0, 0, 0); } + @Override + protected boolean isEndOffsetExclusive() + { + return true; + } + @Override protected boolean isOffsetAtOrBeyond(String current, String target) { @@ -3301,6 +3307,12 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() return false; } + @Override + protected boolean isEndOffsetExclusive() + { + return true; + } + @Override protected boolean isOffsetAtOrBeyond(String current, String target) { @@ -3442,6 +3454,12 @@ protected void scheduleReporting(ScheduledExecutorService reportingExec) this::emitNoticesQueueSize ); } + + @Override + protected boolean isEndOffsetExclusive() + { + return true; + } } private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem 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 9f8625d09cb9..e87c44e2e498 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 @@ -301,6 +301,12 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() { return false; } + + @Override + protected boolean isEndOffsetExclusive() + { + return true; + } } class TestSeekableStreamSupervisor extends BaseTestSeekableStreamSupervisor From cf623b872c1d46b73217e8ad14f49cbbb68fd757 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 5 May 2026 13:23:19 -0700 Subject: [PATCH 37/52] Increase branch coverage --- .../RabbitStreamSupervisorTest.java | 12 +++++++ .../kafka/KafkaDataSourceMetadataTest.java | 32 +++++++++++++++++++ .../SupervisorStateManagerTest.java | 15 +++++++++ 3 files changed, 59 insertions(+) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index 5ff274df357e..82ee7afd0e01 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -548,4 +548,16 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() Assert.assertTrue(supervisor.isOffsetAtOrBeyond(100L, 100L)); Assert.assertFalse(supervisor.isOffsetAtOrBeyond(50L, 100L)); } + + @Test + public void testCreateSequenceOffsetFromObject_invalidType() + { + supervisor = getDefaultSupervisor(); + + Exception e = Assert.assertThrows( + IllegalArgumentException.class, + () -> supervisor.createSequenceOffsetFromObject(new Object()) + ); + Assert.assertTrue(e.getMessage().contains("Cannot convert")); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java index 0b013b69a715..c0faa04d26c0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; import org.apache.druid.initialization.CoreInjectorBuilder; import org.apache.druid.initialization.DruidModule; import org.apache.druid.utils.CollectionUtils; @@ -916,6 +917,37 @@ private static KafkaDataSourceMetadata endMetadataMultiTopic( return new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topicPattern, newOffsets)); } + @Test + public void testEquals_differentSequenceNumbers() + { + // Test equals with different sequence numbers (line 129 false branch) + KafkaDataSourceMetadata metadata1 = startMetadata("foo", ImmutableMap.of(0, 100L)); + KafkaDataSourceMetadata metadata2 = startMetadata("foo", ImmutableMap.of(0, 200L)); + + Assert.assertNotEquals(metadata1, metadata2); + } + + @Test + public void testEquals_sameSequenceNumbers_differentBoundedConfig() + { + // Test equals with same sequence numbers but different boundedStreamConfig (line 130 false branch) + Map boundedStart1 = ImmutableMap.of("0", 0L); + Map boundedEnd1 = ImmutableMap.of("0", 100L); + BoundedStreamConfig boundedConfig1 = new BoundedStreamConfig(boundedStart1, boundedEnd1); + + Map boundedStart2 = ImmutableMap.of("0", 0L); + Map boundedEnd2 = ImmutableMap.of("0", 200L); + BoundedStreamConfig boundedConfig2 = new BoundedStreamConfig(boundedStart2, boundedEnd2); + + SeekableStreamStartSequenceNumbers partitions = + new SeekableStreamStartSequenceNumbers<>("foo", ImmutableMap.of(new KafkaTopicPartition(false, "foo", 0), 100L), ImmutableSet.of()); + + KafkaDataSourceMetadata metadata1 = new KafkaDataSourceMetadata(partitions, boundedConfig1); + KafkaDataSourceMetadata metadata2 = new KafkaDataSourceMetadata(partitions, boundedConfig2); + + Assert.assertNotEquals(metadata1, metadata2); + } + private static ObjectMapper createObjectMapper() { DruidModule module = new KafkaIndexTaskModule(); diff --git a/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java b/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java index 544086ef842c..3f8a38054615 100644 --- a/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java +++ b/server/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerTest.java @@ -123,4 +123,19 @@ public void testCompletedStateIsNotFirstRunOnly() Assert.assertFalse(SupervisorStateManager.BasicState.COMPLETED.isFirstRunOnly()); } + + @Test + public void testMarkRunFinished_completedStateSkipsHealthyCheck() + { + stateManagerConfig = new SupervisorStateManagerConfig(); + SupervisorStateManager supervisorStateManager = new SupervisorStateManager( + stateManagerConfig, + false + ); + + supervisorStateManager.maybeSetState(SupervisorStateManager.BasicState.COMPLETED); + supervisorStateManager.markRunFinished(); + + Assert.assertEquals(SupervisorStateManager.BasicState.COMPLETED, supervisorStateManager.getSupervisorState()); + } } From a037ccb0f59aafad464dafbe1ebcce33f366f847 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 5 May 2026 14:48:21 -0700 Subject: [PATCH 38/52] Increase branch coverage --- .../kafka/KafkaDataSourceMetadataTest.java | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java index c0faa04d26c0..66fe74c9545f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaDataSourceMetadataTest.java @@ -948,6 +948,45 @@ public void testEquals_sameSequenceNumbers_differentBoundedConfig() Assert.assertNotEquals(metadata1, metadata2); } + @Test + public void testEquals_sameSequenceNumbers_sameBoundedConfig() + { + Map boundedStart = ImmutableMap.of("0", 0L); + Map boundedEnd = ImmutableMap.of("0", 100L); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(boundedStart, boundedEnd); + + SeekableStreamStartSequenceNumbers partitions = + new SeekableStreamStartSequenceNumbers<>("foo", ImmutableMap.of(new KafkaTopicPartition(false, "foo", 0), 100L), ImmutableSet.of()); + + KafkaDataSourceMetadata metadata1 = new KafkaDataSourceMetadata(partitions, boundedConfig); + KafkaDataSourceMetadata metadata2 = new KafkaDataSourceMetadata(partitions, boundedConfig); + + Assert.assertEquals(metadata1, metadata2); + } + + @Test + public void testEquals_differentSequenceNumbers_differentBoundedConfig() + { + Map boundedStart1 = ImmutableMap.of("0", 0L); + Map boundedEnd1 = ImmutableMap.of("0", 100L); + BoundedStreamConfig boundedConfig1 = new BoundedStreamConfig(boundedStart1, boundedEnd1); + + Map boundedStart2 = ImmutableMap.of("0", 0L); + Map boundedEnd2 = ImmutableMap.of("0", 200L); + BoundedStreamConfig boundedConfig2 = new BoundedStreamConfig(boundedStart2, boundedEnd2); + + KafkaDataSourceMetadata metadata1 = new KafkaDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>("foo", ImmutableMap.of(new KafkaTopicPartition(false, "foo", 0), 100L), ImmutableSet.of()), + boundedConfig1 + ); + KafkaDataSourceMetadata metadata2 = new KafkaDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>("foo", ImmutableMap.of(new KafkaTopicPartition(false, "foo", 0), 200L), ImmutableSet.of()), + boundedConfig2 + ); + + Assert.assertNotEquals(metadata1, metadata2); + } + private static ObjectMapper createObjectMapper() { DruidModule module = new KafkaIndexTaskModule(); From 0e6466c1f6c9f748581630b57b15ee45e969c46f Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 5 May 2026 17:54:21 -0700 Subject: [PATCH 39/52] Unit test coverage --- .../SeekableStreamSupervisorSpecTest.java | 70 +++++++++++++++++++ 1 file changed, 70 insertions(+) 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 0efad16bc276..dbef4c64ca24 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 @@ -1533,4 +1533,74 @@ private static Map getScaleInProperties() return autoScalerConfig; } + @Test + public void testBoundedStreamSupervisorSpec_runsWithBoundedConfig() + { + org.apache.druid.java.util.emitter.EmittingLogger.registerEmitter(emitter); + + Map startOffsets = ImmutableMap.of("0", 0L); + Map endOffsets = ImmutableMap.of("0", 100L); + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(startOffsets, endOffsets); + + SeekableStreamSupervisorIOConfig boundedIoConfig = new SeekableStreamSupervisorIOConfig( + "stream", + new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), false, false, false), + 1, + 1, + new Period("PT1H"), + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + null, + LagAggregator.DEFAULT, + null, + null, + null, + null, + boundedConfig + ) + { + }; + + EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(boundedIoConfig).anyTimes(); + EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); + EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.replay(spec); + + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(boundedIoConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.absent()).anyTimes(); + EasyMock.replay(taskMaster); + + EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.expect(taskStorage.getActiveTasksByDatasource(EasyMock.anyString())).andReturn(ImmutableList.of()).anyTimes(); + EasyMock.replay(taskStorage); + + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(EasyMock.anyString())).andReturn(null).anyTimes(); + EasyMock.replay(indexerMetadataStorageCoordinator); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(1); + + supervisor.start(); + supervisor.runInternal(); + + // Verify bounded config is properly set + Assert.assertTrue(supervisor.getIoConfig().isBounded()); + Assert.assertNotNull(supervisor.getIoConfig().getBoundedStreamConfig()); + Assert.assertEquals(startOffsets, supervisor.getIoConfig().getBoundedStreamConfig().getStartSequenceNumbers()); + Assert.assertEquals(endOffsets, supervisor.getIoConfig().getBoundedStreamConfig().getEndSequenceNumbers()); + } + } From b1b1179701dca9c55794dbb2f0b4a0d959a37f21 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 5 May 2026 17:55:12 -0700 Subject: [PATCH 40/52] Fix import --- .../supervisor/SeekableStreamSupervisorSpecTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 dbef4c64ca24..895f32d1ac6c 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 @@ -39,6 +39,7 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.metadata.MetadataSupervisorManager; @@ -1536,7 +1537,7 @@ private static Map getScaleInProperties() @Test public void testBoundedStreamSupervisorSpec_runsWithBoundedConfig() { - org.apache.druid.java.util.emitter.EmittingLogger.registerEmitter(emitter); + EmittingLogger.registerEmitter(emitter); Map startOffsets = ImmutableMap.of("0", 0L); Map endOffsets = ImmutableMap.of("0", 100L); From 126638f91d11851e0cc170d61f88a79503817ca9 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 5 May 2026 19:00:19 -0700 Subject: [PATCH 41/52] Remove use of deprecated function --- .../supervisor/SeekableStreamSupervisorSpecTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 895f32d1ac6c..912fd94d5bd3 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 @@ -1568,7 +1568,7 @@ public void testBoundedStreamSupervisorSpec_runsWithBoundedConfig() EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); - EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getSpec().getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(boundedIoConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); From 2ff9cfdc3e351e82191ec6ea23dec19785131163 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 5 May 2026 19:36:01 -0700 Subject: [PATCH 42/52] Revert to deprecated function since not initialized in mock object --- .../supervisor/SeekableStreamSupervisorSpecTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 912fd94d5bd3..895f32d1ac6c 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 @@ -1568,7 +1568,7 @@ public void testBoundedStreamSupervisorSpec_runsWithBoundedConfig() EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); - EasyMock.expect(spec.getSpec().getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(boundedIoConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); From 31c870e8ec1856f21e5f7a4a165d7006d13294a7 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 6 May 2026 11:16:23 -0700 Subject: [PATCH 43/52] Fix merge conflict --- .../indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java | 1 + 1 file changed, 1 insertion(+) 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 2d0f917e529c..a6a01d8640d1 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 @@ -444,6 +444,7 @@ private KafkaSupervisorIOConfig makeIOConfig(Integer taskCount, Map Date: Thu, 7 May 2026 12:59:27 -0700 Subject: [PATCH 44/52] Detect metadata mismatch when committed offset > bounded config end --- .../indexing/KafkaBoundedSupervisorTest.java | 58 +++++++++++++++++-- .../supervisor/SeekableStreamSupervisor.java | 7 +++ 2 files changed, 61 insertions(+), 4 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index 43e74afd221c..a361985c12a2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -183,10 +183,6 @@ public void test_boundedSupervisor_withMismatchedMetadata_is_unhealthy() Assertions.assertEquals("COMPLETED", status1.getState()); // Now try to create a second bounded supervisor with different bounded config on the same datasource - // The key is that the second supervisor's range [50, 200] overlaps with the first supervisor's - // checkpointed offsets (~100), so the metadata mismatch will be detected. If the bounded end is less - // then the checkpointed offset then the Supervisor detects that no work is needed and the Supervisor - // completes silently. Map startOffsets2 = new HashMap<>(); startOffsets2.put("0", 50L); // Different start offset startOffsets2.put("1", 50L); @@ -216,6 +212,60 @@ public void test_boundedSupervisor_withMismatchedMetadata_is_unhealthy() Assertions.assertEquals("UNHEALTHY_SUPERVISOR", status2.getState(), "Supervisor state should be UNHEALTHY_SUPERVISOR"); } + /** + * Regression test: a new bounded run whose endOffset is less than the offset committed by a prior + * run must not silently reach COMPLETED. Before the fix, hasTaskGroupReachedBoundedEnd() compared + * the stale committed offset against the new endOffset (e.g. committed=100 >= newEnd=50) and + * returned true, bypassing task creation and the documented mismatch error entirely. + */ + @Test + public void test_boundedSupervisor_doesNotSilentlyCompleteWhenStaleOffsetExceedsNewEnd() + { + final String topic = IdUtils.getRandomId(); + kafkaServer.createTopicWithPartitions(topic, 2); + publish1kRecords(topic, false); + + // Run 1: ingest up to offset 100 on each partition and complete. + Map startOffsets1 = new HashMap<>(); + startOffsets1.put("0", 0L); + startOffsets1.put("1", 0L); + + Map endOffsets1 = new HashMap<>(); + endOffsets1.put("0", 100L); + endOffsets1.put("1", 100L); + + BoundedStreamConfig boundedConfig1 = new BoundedStreamConfig(startOffsets1, endOffsets1); + final KafkaSupervisorSpec supervisor1 = createBoundedKafkaSupervisor(kafkaServer, topic, boundedConfig1); + + cluster.callApi().postSupervisor(supervisor1); + waitUntilPublishedRecordsAreIngested(200); + waitForSupervisorToComplete(supervisor1.getId()); + + final SupervisorStatus status1 = cluster.callApi().getSupervisorStatus(supervisor1.getId()); + Assertions.assertEquals("COMPLETED", status1.getState()); + + // Run 2: same datasource, endOffset (50) < stale committed offset (100). + // Without the fix the supervisor reaches COMPLETED immediately without running tasks. + // With the fix it detects the config mismatch and becomes UNHEALTHY_SUPERVISOR. + Map startOffsets2 = new HashMap<>(); + startOffsets2.put("0", 0L); + startOffsets2.put("1", 0L); + + Map endOffsets2 = new HashMap<>(); + endOffsets2.put("0", 50L); + endOffsets2.put("1", 50L); + + BoundedStreamConfig boundedConfig2 = new BoundedStreamConfig(startOffsets2, endOffsets2); + final KafkaSupervisorSpec supervisor2 = createBoundedKafkaSupervisor(kafkaServer, topic, boundedConfig2); + + cluster.callApi().postSupervisor(supervisor2); + waitForSupervisorToBeUnhealthy(supervisor2.getId()); + + final SupervisorStatus status2 = cluster.callApi().getSupervisorStatus(supervisor2.getId()); + Assertions.assertFalse(status2.isHealthy(), "Supervisor should be unhealthy after detecting metadata mismatch"); + Assertions.assertEquals("UNHEALTHY_SUPERVISOR", status2.getState(), "Supervisor state should be UNHEALTHY_SUPERVISOR"); + } + private void waitForSupervisorToComplete(String supervisorId) { // Wait for supervisor to reach COMPLETED state 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 5bad59e5c8f4..57f3a603806a 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 @@ -4601,6 +4601,13 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) return true; } + // Offsets from a prior bounded run with a different config must not count as completion evidence. + // Returning false lets createNewTasks() run, which calls getOffsetFromStorageForPartition() + // and throws the documented mismatch error. + BoundedStreamConfig metadataBoundedConfig = getBoundedConfigFromMetadata(retrieveDataSourceMetadata()); + if (!boundedConfig.equals(metadataBoundedConfig)) { + return false; + } Map currentOffsets = getOffsetsFromMetadataStorage(); log.info( From 6c33c8ff68ce6487a36c8b0c8bbfe6901e52b808 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 12 May 2026 15:19:57 -0700 Subject: [PATCH 45/52] Clean up redundant tests in BoundedStreamConfigTest and use EqualsVerifier --- .../supervisor/BoundedStreamConfigTest.java | 226 ++---------------- 1 file changed, 19 insertions(+), 207 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java index 8eea77e58ad6..a29d63add00b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/BoundedStreamConfigTest.java @@ -20,11 +20,11 @@ package org.apache.druid.indexing.seekablestream.supervisor; import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.error.DruidException; import org.junit.Assert; import org.junit.Test; -import java.util.HashMap; import java.util.Map; public class BoundedStreamConfigTest @@ -32,139 +32,85 @@ public class BoundedStreamConfigTest private final ObjectMapper mapper = new ObjectMapper(); @Test - public void testConstructorWithValidMaps() + public void testEqualsAndHashCode() { - Map startOffsets = new HashMap<>(); - startOffsets.put("0", 100L); - startOffsets.put("1", 200L); - - Map endOffsets = new HashMap<>(); - endOffsets.put("0", 500L); - endOffsets.put("1", 600L); - - BoundedStreamConfig config = new BoundedStreamConfig(startOffsets, endOffsets); - - Assert.assertEquals(startOffsets, config.getStartSequenceNumbers()); - Assert.assertEquals(endOffsets, config.getEndSequenceNumbers()); + EqualsVerifier.forClass(BoundedStreamConfig.class) + .withNonnullFields("startSequenceNumbers", "endSequenceNumbers") + .usingGetClass() + .verify(); } @Test public void testConstructorWithNullStartSequenceNumbers() { - Map endOffsets = new HashMap<>(); - endOffsets.put("0", 500L); - DruidException ex = Assert.assertThrows( DruidException.class, - () -> new BoundedStreamConfig(null, endOffsets) + () -> new BoundedStreamConfig(null, Map.of("0", 500L)) ); - Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); } @Test public void testConstructorWithNullEndSequenceNumbers() { - Map startOffsets = new HashMap<>(); - startOffsets.put("0", 100L); - DruidException ex = Assert.assertThrows( DruidException.class, - () -> new BoundedStreamConfig(startOffsets, null) + () -> new BoundedStreamConfig(Map.of("0", 100L), null) ); - Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); } @Test public void testConstructorWithEmptyStartSequenceNumbers() { - Map startOffsets = new HashMap<>(); - Map endOffsets = new HashMap<>(); - endOffsets.put("0", 500L); - DruidException ex = Assert.assertThrows( DruidException.class, - () -> new BoundedStreamConfig(startOffsets, endOffsets) + () -> new BoundedStreamConfig(Map.of(), Map.of("0", 500L)) ); - Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); } @Test public void testConstructorWithEmptyEndSequenceNumbers() { - Map startOffsets = new HashMap<>(); - startOffsets.put("0", 100L); - Map endOffsets = new HashMap<>(); - DruidException ex = Assert.assertThrows( DruidException.class, - () -> new BoundedStreamConfig(startOffsets, endOffsets) + () -> new BoundedStreamConfig(Map.of("0", 100L), Map.of()) ); - Assert.assertTrue(ex.getMessage().contains("cannot be null or empty")); } @Test public void testConstructorWithMismatchedPartitions() { - Map startOffsets = new HashMap<>(); - startOffsets.put("0", 100L); - Map endOffsets = new HashMap<>(); - endOffsets.put("1", 500L); - DruidException ex = Assert.assertThrows( DruidException.class, - () -> new BoundedStreamConfig(startOffsets, endOffsets) + () -> new BoundedStreamConfig(Map.of("0", 100L), Map.of("1", 500L)) ); - Assert.assertTrue(ex.getMessage().contains("must have matching partition sets")); } @Test public void testSerializationDeserialization() throws Exception { - Map startOffsets = new HashMap<>(); - startOffsets.put("0", 100); - startOffsets.put("1", 200); - - Map endOffsets = new HashMap<>(); - endOffsets.put("0", 500); - endOffsets.put("1", 600); - - BoundedStreamConfig config = new BoundedStreamConfig(startOffsets, endOffsets); + BoundedStreamConfig config = new BoundedStreamConfig( + Map.of("0", 100, "1", 200), + Map.of("0", 500, "1", 600) + ); - String json = mapper.writeValueAsString(config); - BoundedStreamConfig deserialized = mapper.readValue(json, BoundedStreamConfig.class); + BoundedStreamConfig deserialized = mapper.readValue( + mapper.writeValueAsString(config), + BoundedStreamConfig.class + ); - // Check sizes Assert.assertEquals(2, deserialized.getStartSequenceNumbers().size()); Assert.assertEquals(2, deserialized.getEndSequenceNumbers().size()); - - // Check that deserialized maps contain expected values (keys will be Strings after deserialization) Assert.assertEquals(100, deserialized.getStartSequenceNumbers().get("0")); Assert.assertEquals(200, deserialized.getStartSequenceNumbers().get("1")); Assert.assertEquals(500, deserialized.getEndSequenceNumbers().get("0")); Assert.assertEquals(600, deserialized.getEndSequenceNumbers().get("1")); } - @Test - public void testDeserializationWithIntegerValues() throws Exception - { - String json = "{" - + "\"startSequenceNumbers\": {\"0\": 100, \"1\": 200}," - + "\"endSequenceNumbers\": {\"0\": 500, \"1\": 600}" - + "}"; - - BoundedStreamConfig config = mapper.readValue(json, BoundedStreamConfig.class); - - Assert.assertNotNull(config.getStartSequenceNumbers()); - Assert.assertNotNull(config.getEndSequenceNumbers()); - Assert.assertEquals(2, config.getStartSequenceNumbers().size()); - Assert.assertEquals(2, config.getEndSequenceNumbers().size()); - } - @Test public void testDeserializationWithStringValues() throws Exception { @@ -175,8 +121,6 @@ public void testDeserializationWithStringValues() throws Exception BoundedStreamConfig config = mapper.readValue(json, BoundedStreamConfig.class); - Assert.assertNotNull(config.getStartSequenceNumbers()); - Assert.assertNotNull(config.getEndSequenceNumbers()); Assert.assertEquals(2, config.getStartSequenceNumbers().size()); Assert.assertEquals(2, config.getEndSequenceNumbers().size()); } @@ -191,139 +135,7 @@ public void testDeserializationWithMixedTypes() throws Exception BoundedStreamConfig config = mapper.readValue(json, BoundedStreamConfig.class); - Assert.assertNotNull(config.getStartSequenceNumbers()); - Assert.assertNotNull(config.getEndSequenceNumbers()); Assert.assertEquals(2, config.getStartSequenceNumbers().size()); Assert.assertEquals(2, config.getEndSequenceNumbers().size()); } - - @Test - public void testEquals_equalObjects() - { - Map start1 = new HashMap<>(); - start1.put("0", 0L); - Map end1 = new HashMap<>(); - end1.put("0", 100L); - - Map start2 = new HashMap<>(); - start2.put("0", 0L); - Map end2 = new HashMap<>(); - end2.put("0", 100L); - - BoundedStreamConfig config1 = new BoundedStreamConfig(start1, end1); - BoundedStreamConfig config2 = new BoundedStreamConfig(start2, end2); - - Assert.assertEquals(config1, config2); - Assert.assertEquals(config1.hashCode(), config2.hashCode()); - } - - @Test - public void testEquals_nullObject() - { - Map start = new HashMap<>(); - start.put("0", 0L); - Map end = new HashMap<>(); - end.put("0", 100L); - - BoundedStreamConfig config = new BoundedStreamConfig(start, end); - - Assert.assertNotEquals(config, null); - } - - @Test - public void testEquals_differentClass() - { - Map start = new HashMap<>(); - start.put("0", 0L); - Map end = new HashMap<>(); - end.put("0", 100L); - - BoundedStreamConfig config = new BoundedStreamConfig(start, end); - - Assert.assertNotEquals(config, "not a BoundedStreamConfig"); - } - - @Test - public void testEquals_differentStartOffsets() - { - Map start1 = new HashMap<>(); - start1.put("0", 0L); - Map start2 = new HashMap<>(); - start2.put("0", 10L); - Map end = new HashMap<>(); - end.put("0", 100L); - - BoundedStreamConfig config1 = new BoundedStreamConfig(start1, end); - BoundedStreamConfig config2 = new BoundedStreamConfig(start2, end); - - Assert.assertNotEquals(config1, config2); - } - - @Test - public void testEquals_differentEndOffsets() - { - Map start = new HashMap<>(); - start.put("0", 0L); - Map end1 = new HashMap<>(); - end1.put("0", 100L); - Map end2 = new HashMap<>(); - end2.put("0", 200L); - - BoundedStreamConfig config1 = new BoundedStreamConfig(start, end1); - BoundedStreamConfig config2 = new BoundedStreamConfig(start, end2); - - Assert.assertNotEquals(config1, config2); - } - - @Test - public void testHashCode_consistency() - { - Map start = new HashMap<>(); - start.put("0", 0L); - Map end = new HashMap<>(); - end.put("0", 100L); - - BoundedStreamConfig config = new BoundedStreamConfig(start, end); - - int hashCode1 = config.hashCode(); - int hashCode2 = config.hashCode(); - - Assert.assertEquals(hashCode1, hashCode2); - } - - @Test - public void testHashCode_equalObjectsSameHashCode() - { - Map start1 = new HashMap<>(); - start1.put("0", 0L); - Map end1 = new HashMap<>(); - end1.put("0", 100L); - - Map start2 = new HashMap<>(); - start2.put("0", 0L); - Map end2 = new HashMap<>(); - end2.put("0", 100L); - - BoundedStreamConfig config1 = new BoundedStreamConfig(start1, end1); - BoundedStreamConfig config2 = new BoundedStreamConfig(start2, end2); - - Assert.assertEquals(config1.hashCode(), config2.hashCode()); - } - - @Test - public void testToString() - { - Map start = new HashMap<>(); - start.put("0", 0L); - Map end = new HashMap<>(); - end.put("0", 100L); - - BoundedStreamConfig config = new BoundedStreamConfig(start, end); - String str = config.toString(); - - Assert.assertNotNull(str); - Assert.assertTrue(str.contains("BoundedStreamConfig")); - Assert.assertTrue(str.contains("startSequenceNumbers")); - Assert.assertTrue(str.contains("endSequenceNumbers")); - } } From 41a31a1912881450d8fdf098c6d2f492f0e3b2e4 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 12 May 2026 15:26:39 -0700 Subject: [PATCH 46/52] Compare Kinesis Sequence numbers using built in comparison --- .../indexing/KafkaBoundedSupervisorTest.java | 10 ++++------ .../kinesis/supervisor/KinesisSupervisor.java | 20 +------------------ .../supervisor/SeekableStreamSupervisor.java | 4 ++-- 3 files changed, 7 insertions(+), 27 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index a361985c12a2..aa772b0f1437 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -213,10 +213,8 @@ public void test_boundedSupervisor_withMismatchedMetadata_is_unhealthy() } /** - * Regression test: a new bounded run whose endOffset is less than the offset committed by a prior - * run must not silently reach COMPLETED. Before the fix, hasTaskGroupReachedBoundedEnd() compared - * the stale committed offset against the new endOffset (e.g. committed=100 >= newEnd=50) and - * returned true, bypassing task creation and the documented mismatch error entirely. + * A new bounded run whose endOffset is less than the offset committed by a prior + * run must not silently reach COMPLETED. */ @Test public void test_boundedSupervisor_doesNotSilentlyCompleteWhenStaleOffsetExceedsNewEnd() @@ -232,13 +230,13 @@ public void test_boundedSupervisor_doesNotSilentlyCompleteWhenStaleOffsetExceeds Map endOffsets1 = new HashMap<>(); endOffsets1.put("0", 100L); - endOffsets1.put("1", 100L); + endOffsets1.put("1", 150L); BoundedStreamConfig boundedConfig1 = new BoundedStreamConfig(startOffsets1, endOffsets1); final KafkaSupervisorSpec supervisor1 = createBoundedKafkaSupervisor(kafkaServer, topic, boundedConfig1); cluster.callApi().postSupervisor(supervisor1); - waitUntilPublishedRecordsAreIngested(200); + waitUntilPublishedRecordsAreIngested(150); waitForSupervisorToComplete(supervisor1.getId()); final SupervisorStatus status1 = cluster.callApi().getSupervisorStatus(supervisor1.getId()); 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 2a9df4052cc9..0f91fc0965db 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 @@ -53,7 +53,6 @@ import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -61,7 +60,6 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.math.BigInteger; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -390,23 +388,7 @@ protected boolean isShardExpirationMarker(String seqNum) @Override protected boolean isOffsetAtOrBeyond(String current, String target) { - // Kinesis sequence numbers are decimal numeric strings that must be compared numerically. - // Use BigInteger because Kinesis sequence numbers can be very large (128-bit). - try { - BigInteger currentNum = new BigInteger(current); - BigInteger targetNum = new BigInteger(target); - return currentNum.compareTo(targetNum) >= 0; - } - catch (NumberFormatException e) { - throw new IAE( - StringUtils.format( - "Invalid Kinesis sequence number. Expected numeric string but got current=[%s], target=[%s]", - current, - target - ), - e - ); - } + return KinesisSequenceNumber.of(current).compareTo(KinesisSequenceNumber.of(target)) >= 0; } @Override 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 57f3a603806a..1d1771ccbbc7 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 @@ -4679,9 +4679,9 @@ private Map getEndOffsetsForGroup(int group /** * For bounded supervisors, we determine completion by checking if new tasks would be created. - * In createNewTasks(), bounded mode checks hasTaskGroupReachedBoundedEnd() before creating tasks. + * In {@link #createNewTasks()}, bounded mode checks {@link #hasTaskGroupReachedBoundedEnd(int)} before creating tasks. * If that returns true (offsets reached), no new tasks are created. - * So completion is: no active tasks, no pending tasks, and createNewTasks() chose not to create any. + * So completion is: no active tasks, no pending tasks, and {@link #createNewTasks()} chose not to create any. * * @return true if all bounded work is complete, false otherwise */ From dd9bdd2b6e4a1c4c906bc663911e76ffb7e146f8 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 12 May 2026 18:21:36 -0700 Subject: [PATCH 47/52] Clean up docs based on review comments --- docs/ingestion/supervisor.md | 4 ++-- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 4 +--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/docs/ingestion/supervisor.md b/docs/ingestion/supervisor.md index 980e763e098e..4579532e09d7 100644 --- a/docs/ingestion/supervisor.md +++ b/docs/ingestion/supervisor.md @@ -65,7 +65,7 @@ For configuration properties specific to Kafka and Kinesis, see [Kafka I/O confi |`lateMessageRejectionPeriod`|ISO 8601 period|Configures tasks to reject messages with timestamps earlier than this period before the task was created. For example, if this property is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a streaming and a nightly batch ingestion pipeline. You can specify only one of the late message rejection properties.|No|| |`earlyMessageRejectionPeriod`|ISO 8601 period|Configures tasks to reject messages with timestamps later than this period after the task reached its task duration. For example, if this property is set to `PT1H`, the task duration is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps later than `2016-01-01T14:00Z`. Tasks sometimes run past their task duration, such as in cases of supervisor failover.|No|| |`stopTaskCount`|Integer|Limits the number of ingestion tasks Druid can cycle at any given time. If not set, Druid can cycle all tasks at the same time. If set to a value less than `taskCount`, your cluster needs fewer available slots to run the supervisor. You can save costs by scaling down your ingestion tier, but this can lead to slower cycle times and lag. See [`stopTaskCount`](#stoptaskcount) for more information.|No|`taskCount` value| -|`boundedStreamConfig`|Object|Configures the supervisor for bounded (one-time) ingestion with explicit start and end offsets. When set, the supervisor creates tasks that read from `startSequenceNumbers` to `endSequenceNumbers`, then automatically terminates when all data is ingested. The bounded configuration is stored with datasource metadata; if a supervisor is restarted or a new supervisor is created with different offsets for the same datasource, it will fail. To retry with different offsets, use the supervisor reset API to clear metadata or use a different datasource. Useful for backfills and historical reprocessing. See [Bounded stream configuration](#bounded-stream-configuration) for details.|No|null| +|`boundedStreamConfig`|Object|Configures the supervisor for bounded (one-time) ingestion with explicit start and end offsets. When set, the supervisor creates tasks that read from `startSequenceNumbers` to `endSequenceNumbers`, then automatically terminates when all data is ingested. The bounded configuration is stored with datasource metadata; if a supervisor is restarted or a new supervisor is created with different offsets for the same datasource, it will fail. To retry with different offsets, use the supervisor reset API to clear metadata or use a different supervisor ID. Useful for backfills and historical reprocessing. See [Bounded stream configuration](#bounded-stream-configuration) for details.|No|null| |`serverPriorityToReplicas`|Object (`Map`)|Map of server priorities to the number of replicas per priority. When set, each task replica is assigned a server priority that corresponds to `druid.server.priority` on the Peon process to enable query isolation for mixed workloads using [query routing strategies](../configuration/index.md#query-routing). If not configured, the `replicas` setting applies and all task replicas are assigned a default priority of 0.

For example, setting `serverPriorityToReplicas` to `{"1": 2, "0": 1}` creates 2 task replicas with `druid.server.priority=1` and 1 task replica with `druid.server.priority=0` per task group. This configuration scales proportionally with `taskCount`. For example, if `taskCount` is set to 5, this results in 15 total tasks - 10 tasks with priority 1 and 5 tasks with priority 0. If both `replicas` and `serverPriorityToReplicas` are set, the sum of replicas in `serverPriorityToReplicas` must equal `replicas`.|No|null| #### Task autoscaler @@ -271,7 +271,7 @@ When configured, the supervisor: **Metadata consistency:** The bounded configuration is stored in datasource metadata along with checkpointed offsets. If you restart the supervisor or create a new supervisor with a different `boundedStreamConfig` for the same datasource, the supervisor will fail with an error. To start a new bounded ingestion with different offsets, either: - Use the [supervisor reset API](../api-reference/supervisor-api.md#reset-a-supervisor) to clear existing metadata -- Use a different datasource name +- Use a different supervisor ID **Example (Kafka):** 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 1d1771ccbbc7..3ce405e68e59 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 @@ -4450,7 +4450,7 @@ private OrderedSequenceNumber getOffsetFromStorageForPartiti + "Metadata bounded config [%s] does not match current config [%s]. " + "To start a new bounded ingestion, either: " + "(1) use the supervisor reset API to clear existing metadata, or " - + "(2) use a different supervisor ID / datasource.", + + "(2) use a different supervisor ID.", metadataBoundedConfig, currentBoundedConfig ); @@ -4602,8 +4602,6 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) } // Offsets from a prior bounded run with a different config must not count as completion evidence. - // Returning false lets createNewTasks() run, which calls getOffsetFromStorageForPartition() - // and throws the documented mismatch error. BoundedStreamConfig metadataBoundedConfig = getBoundedConfigFromMetadata(retrieveDataSourceMetadata()); if (!boundedConfig.equals(metadataBoundedConfig)) { return false; From fd7981f444535b73895f70114d1d2a94df24e936 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 12 May 2026 18:35:24 -0700 Subject: [PATCH 48/52] Early return before convert for hasTaskGroupReachedBoundedEnd --- .../embedded/indexing/KafkaBoundedSupervisorTest.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 9 ++++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index aa772b0f1437..2fb5018eca81 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -236,7 +236,7 @@ public void test_boundedSupervisor_doesNotSilentlyCompleteWhenStaleOffsetExceeds final KafkaSupervisorSpec supervisor1 = createBoundedKafkaSupervisor(kafkaServer, topic, boundedConfig1); cluster.callApi().postSupervisor(supervisor1); - waitUntilPublishedRecordsAreIngested(150); + waitUntilPublishedRecordsAreIngested(250); waitForSupervisorToComplete(supervisor1.getId()); final SupervisorStatus status1 = cluster.callApi().getSupervisorStatus(supervisor1.getId()); 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 3ce405e68e59..1f2d7eb5ba0d 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 @@ -4555,17 +4555,16 @@ && checkSourceMetadataMatch(dataSourceMetadata)) { */ private boolean hasTaskGroupReachedBoundedEnd(int groupId) { + Set partitionsInGroup = partitionGroups.get(groupId); + if (partitionsInGroup == null || partitionsInGroup.isEmpty()) { + return false; + } BoundedStreamConfig boundedConfig = ioConfig.getBoundedStreamConfig(); Map startOffsets = convertBoundedConfigMap(boundedConfig.getStartSequenceNumbers()); Map endOffsets = convertBoundedConfigMap(boundedConfig.getEndSequenceNumbers()); - Set partitionsInGroup = partitionGroups.get(groupId); - if (partitionsInGroup == null || partitionsInGroup.isEmpty()) { - return false; - } - // Check if all partitions have empty ranges // For exclusive end offsets (Kafka): start >= end means empty // For inclusive end offsets (Kinesis): only start > end means empty (start == end is one record) From cc13428b3e6f21eb9abf3f8d9145a6f4c4e11c44 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Tue, 12 May 2026 18:47:50 -0700 Subject: [PATCH 49/52] Resolve merge conflicts --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 bd13a6712fff..41f82daa2c1f 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 @@ -4317,7 +4317,6 @@ private void createNewTasks() throws JsonProcessingException simpleStartingOffsets, endOffsets ); - activelyReadingTaskGroups.put( newTaskGroups.put( groupId, new TaskGroup( @@ -4419,7 +4418,6 @@ private Map> generate // Existing logic for both streaming and bounded mode ImmutableMap.Builder> builder = ImmutableMap.builder(); final DataSourceMetadata dataSourceMetadata = retrieveDataSourceMetadata(); - final Map metadataOffsets = getOffsetsFromMetadataStorage(); final BoundedStreamConfig metadataBoundedConfig = getBoundedConfigFromMetadata(dataSourceMetadata); for (PartitionIdType partitionId : partitionGroups.get(groupId)) { @@ -4436,7 +4434,7 @@ private Map> generate OrderedSequenceNumber offsetFromStorage = getOffsetFromStorageForPartition( partitionId, metadataOffsets, - metadataBoundedConfig + metadataBoundedConfig, partitionsToReset ); @@ -4473,7 +4471,7 @@ private BoundedStreamConfig getBoundedConfigFromMetadata(@Nullable DataSourceMet private OrderedSequenceNumber getOffsetFromStorageForPartition( PartitionIdType partition, final Map metadataOffsets, - @Nullable final BoundedStreamConfig metadataBoundedConfig + @Nullable final BoundedStreamConfig metadataBoundedConfig, final Map partitionsToReset ) { From c62b93608b58327f3fb7faa1bf8797c439a4a08c Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 13 May 2026 12:03:42 -0700 Subject: [PATCH 50/52] Fix KinesisSupervisorTest --- .../indexing/KafkaBoundedSupervisorTest.java | 92 +++++-------------- .../supervisor/KinesisSupervisorTest.java | 11 +-- 2 files changed, 27 insertions(+), 76 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index 2fb5018eca81..a7d585cfc4e2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -25,7 +25,8 @@ import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; import org.apache.druid.indexing.seekablestream.supervisor.BoundedStreamConfig; -import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; import org.apache.druid.testing.embedded.StreamIngestResource; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -38,7 +39,6 @@ */ public class KafkaBoundedSupervisorTest extends StreamIndexTestBase { - private static final EmittingLogger log = new EmittingLogger(KafkaBoundedSupervisorTest.class); private final KafkaResource kafkaServer = new KafkaResource(); @Override @@ -47,6 +47,17 @@ protected StreamIngestResource getStreamIngestResource() return kafkaServer; } + @Override + protected EmbeddedDruidCluster createCluster() + { + overlord.addProperty( + "druid.monitoring.monitors", + "[\"org.apache.druid.server.metrics.SupervisorStatsMonitor\"]" + ); + overlord.addProperty("druid.monitoring.emissionPeriod", "PT1s"); + return super.createCluster(); + } + @Test public void test_boundedSupervisor_ingestsDataAndCompletes() { @@ -203,7 +214,6 @@ public void test_boundedSupervisor_withMismatchedMetadata_is_unhealthy() cluster.callApi().postSupervisor(supervisor2); // Wait for the supervisor to process and detect the metadata mismatch - // The exception we're testing for is thrown and logged, and causes the supervisor to become unhealthy waitForSupervisorToBeUnhealthy(supervisor2.getId()); // Verify the supervisor is unhealthy @@ -266,75 +276,19 @@ public void test_boundedSupervisor_doesNotSilentlyCompleteWhenStaleOffsetExceeds private void waitForSupervisorToComplete(String supervisorId) { - // Wait for supervisor to reach COMPLETED state - int maxAttempts = 60; // 60 seconds timeout - int attempt = 0; - - while (attempt < maxAttempts) { - try { - SupervisorStatus status = cluster.callApi().getSupervisorStatus(supervisorId); - if ("COMPLETED".equals(status.getState())) { - return; - } - Thread.sleep(1000); - attempt++; - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting for supervisor to complete", e); - } - catch (Exception e) { - // Supervisor might not be found immediately, retry - attempt++; - try { - Thread.sleep(1000); - } - catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting", ie); - } - } - } - - Assertions.fail("Supervisor did not complete within timeout"); + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("supervisor/count") + .hasDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) + .hasDimension("state", "COMPLETED") + ); } private void waitForSupervisorToBeUnhealthy(String supervisorId) { - // Wait for supervisor to become unhealthy after detecting the metadata mismatch - int maxAttempts = 30; // 30 seconds timeout - int attempt = 0; - - while (attempt < maxAttempts) { - try { - SupervisorStatus status = cluster.callApi().getSupervisorStatus(supervisorId); - - // The supervisor should become unhealthy when the exception is thrown - if (!status.isHealthy()) { - log.info("Supervisor became unhealthy with state: %s", status.getDetailedState()); - return; - } - - Thread.sleep(1000); - attempt++; - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting for supervisor to become unhealthy", e); - } - catch (Exception e) { - // Supervisor might not be found immediately, retry - attempt++; - try { - Thread.sleep(1000); - } - catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting", ie); - } - } - } - - Assertions.fail("Supervisor did not become unhealthy due to metadata mismatch within timeout"); + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("supervisor/count") + .hasDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) + .hasDimension("state", "UNHEALTHY_SUPERVISOR") + ); } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 7602df1b9d61..79e983ae4c4f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -4824,16 +4824,13 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() } @Test - public void testIsOffsetAtOrBeyond_invalidSequenceNumber() + public void testIsOffsetAtOrBeyond_nonNumericSequenceNumber() { supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); - Exception e = Assert.assertThrows( - IllegalArgumentException.class, - () -> supervisor.isOffsetAtOrBeyond("not-a-number", "12345") - ); - Assert.assertTrue(e.getMessage().contains("Invalid Kinesis sequence number")); - Assert.assertTrue(e.getMessage().contains("not-a-number")); + // Non-numeric strings (e.g. EOS, EXPIRED) are treated as max sequence numbers by + // KinesisSequenceNumber, so they are considered at or beyond any numeric offset. + Assert.assertTrue(supervisor.isOffsetAtOrBeyond("not-a-number", "12345")); } @Test From 35c4574e75f398d79d1125550f94dc82846dcc0d Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 13 May 2026 14:02:31 -0700 Subject: [PATCH 51/52] Update KinesisSupervisorTest.java --- .../kinesis/supervisor/KinesisSupervisorTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 79e983ae4c4f..3f6da8aa0532 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -4824,13 +4824,13 @@ public void testBoundedModeCreateTasksWithCorrectOffsets() } @Test - public void testIsOffsetAtOrBeyond_nonNumericSequenceNumber() + public void testIsOffsetAtOrBeyond_specialMarkerIsAtOrBeyondNumeric() { supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); - // Non-numeric strings (e.g. EOS, EXPIRED) are treated as max sequence numbers by - // KinesisSequenceNumber, so they are considered at or beyond any numeric offset. - Assert.assertTrue(supervisor.isOffsetAtOrBeyond("not-a-number", "12345")); + // Special markers like EOS are treated as max sequence numbers by KinesisSequenceNumber, + // so they are considered at or beyond any numeric offset. + Assert.assertTrue(supervisor.isOffsetAtOrBeyond(KinesisSequenceNumber.END_OF_SHARD_MARKER, "12345")); } @Test From e0b5ef971df363bd3e6dfe84ca9d74c3f2090310 Mon Sep 17 00:00:00 2001 From: Andrew Ho Date: Wed, 13 May 2026 14:46:14 -0700 Subject: [PATCH 52/52] Cover case where start > end --- .../indexing/KafkaBoundedSupervisorTest.java | 18 +++++++++++++++ .../supervisor/SeekableStreamSupervisor.java | 22 +++++++++---------- 2 files changed, 28 insertions(+), 12 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java index a7d585cfc4e2..7e22d85d9cab 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaBoundedSupervisorTest.java @@ -138,6 +138,24 @@ public void test_boundedSupervisor_withEmptyRange_completesImmediately() Assertions.assertEquals("COMPLETED", status.getState()); } + @Test + public void test_boundedSupervisor_withReversedRange_isUnhealthy() + { + final String topic = IdUtils.getRandomId(); + kafkaServer.createTopicWithPartitions(topic, 1); + + // start > end — invalid range, KafkaIndexTaskIOConfig rejects it when a task is created. + BoundedStreamConfig boundedConfig = new BoundedStreamConfig(Map.of("0", 500L), Map.of("0", 100L)); + final KafkaSupervisorSpec supervisor = createBoundedKafkaSupervisor(kafkaServer, topic, boundedConfig); + + cluster.callApi().postSupervisor(supervisor); + waitForSupervisorToBeUnhealthy(supervisor.getId()); + + final SupervisorStatus status = cluster.callApi().getSupervisorStatus(supervisor.getId()); + Assertions.assertFalse(status.isHealthy()); + Assertions.assertEquals("UNHEALTHY_SUPERVISOR", status.getState()); + } + private KafkaSupervisorSpec createBoundedKafkaSupervisor( KafkaResource kafkaServer, String topic, 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 41f82daa2c1f..91b4244c0bf3 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 @@ -4596,23 +4596,22 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) Map endOffsets = convertBoundedConfigMap(boundedConfig.getEndSequenceNumbers()); - // Check if all partitions have empty ranges - // For exclusive end offsets (Kafka): start >= end means empty - // For inclusive end offsets (Kinesis): only start > end means empty (start == end is one record) + // Check if all partitions have empty ranges. + // Reversed ranges (start > end) return false so createNewTasks() runs and the task IO config + // rejects the invalid range with a clear error, making the supervisor UNHEALTHY. + // For exclusive end offsets (Kafka): start == end is a legitimate empty range. + // For inclusive end offsets (Kinesis): start == end is one record, not empty. boolean allPartitionsEmptyRange = true; for (PartitionIdType partition : partitionsInGroup) { SequenceOffsetType start = startOffsets.get(partition); SequenceOffsetType end = endOffsets.get(partition); - boolean isEmpty; - if (isEndOffsetExclusive()) { - // Exclusive: empty if start >= end - isEmpty = isOffsetAtOrBeyond(start, end); - } else { - // Inclusive: empty only if start > end - isEmpty = isOffsetAtOrBeyond(start, end) && !start.equals(end); + boolean startBeyondEnd = isOffsetAtOrBeyond(start, end) && !start.equals(end); + if (startBeyondEnd) { + return false; } + boolean isEmpty = isEndOffsetExclusive() && start.equals(end); if (!isEmpty) { allPartitionsEmptyRange = false; break; @@ -4621,10 +4620,9 @@ private boolean hasTaskGroupReachedBoundedEnd(int groupId) if (allPartitionsEmptyRange) { log.warn( - "TaskGroup[%d] has empty range for all partitions (start %s end). " + "TaskGroup[%d] has empty range for all partitions (start == end). " + "No work to do, marking as complete. Start: %s, End: %s", groupId, - isEndOffsetExclusive() ? ">=" : ">", startOffsets, endOffsets );