From 8d683ab5d79f20970c95c5672659c289b52f6d05 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Tue, 2 Sep 2025 10:45:46 +0530 Subject: [PATCH 01/46] task partition reassignment API design --- .../seekablestream/PartitionAssignment.java | 96 +++++++++++++++++++ .../PartitionReassignmentRequest.java | 76 +++++++++++++++ .../SeekableStreamIndexTaskRunner.java | 77 +++++++++++++++ 3 files changed, 249 insertions(+) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionReassignmentRequest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java new file mode 100644 index 000000000000..f8e6214749ef --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java @@ -0,0 +1,96 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * Represents a single partition assignment with its start and end offsets. + */ +public class PartitionAssignment +{ + private final PartitionIdType partitionId; + private final SequenceOffsetType startOffset; + private final SequenceOffsetType endOffset; + + @JsonCreator + public PartitionAssignment( + @JsonProperty("partitionId") PartitionIdType partitionId, + @JsonProperty("startOffset") SequenceOffsetType startOffset, + @JsonProperty("endOffset") SequenceOffsetType endOffset + ) + { + this.partitionId = partitionId; + this.startOffset = startOffset; + this.endOffset = endOffset; + } + + @JsonProperty + public PartitionIdType getPartitionId() + { + return partitionId; + } + + @JsonProperty + public SequenceOffsetType getStartOffset() + { + return startOffset; + } + + @JsonProperty + public SequenceOffsetType getEndOffset() + { + return endOffset; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionAssignment that = (PartitionAssignment) o; + return Objects.equals(partitionId, that.partitionId) && + Objects.equals(startOffset, that.startOffset) && + Objects.equals(endOffset, that.endOffset); + } + + @Override + public int hashCode() + { + return Objects.hash(partitionId, startOffset, endOffset); + } + + @Override + public String toString() + { + return "PartitionAssignment{" + + "partitionId=" + partitionId + + ", startOffset=" + startOffset + + ", endOffset=" + endOffset + + '}'; + } +} \ No newline at end of file diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionReassignmentRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionReassignmentRequest.java new file mode 100644 index 000000000000..66c30d606d8c --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionReassignmentRequest.java @@ -0,0 +1,76 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +/** + * Request object for reassigning partitions to a seekable stream index task. + * Contains a list of partition assignments, each with partition ID, start offset, and end offset. + */ +public class PartitionReassignmentRequest +{ + private final List> partitionAssignments; + + @JsonCreator + public PartitionReassignmentRequest( + @JsonProperty("partitionAssignments") List> partitionAssignments + ) + { + this.partitionAssignments = partitionAssignments; + } + + @JsonProperty + public List> getPartitionAssignments() + { + return partitionAssignments; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionReassignmentRequest that = (PartitionReassignmentRequest) o; + return Objects.equals(partitionAssignments, that.partitionAssignments); + } + + @Override + public int hashCode() + { + return Objects.hash(partitionAssignments); + } + + @Override + public String toString() + { + return "PartitionReassignmentRequest{" + + "partitionAssignments=" + partitionAssignments + + '}'; + } +} \ No newline at end of file diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index c6e4a9ff2356..0d32b8876fd9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1717,6 +1717,83 @@ public Response getUnparseableEvents( return Response.ok(parseExceptionHandler.getSavedParseExceptionReports()).build(); } + @POST + @Path("/updateAssignments") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public void reassignPartitions(PartitionReassignmentRequest req) throws InterruptedException + { + try { + requestPause(); + checkpointSequences(); + createNewSequenceForPartitions(req.getPartitionAssignments()); + resume(); + } catch (Exception e) { + log.makeAlert(e, "Failed to reassign partitions."); + } + } + + private void createNewSequenceForPartitions(List> newPartitions) + throws IOException + { + Map newPartitionStartOffsets = new HashMap<>(); + Map newPartitionEndOffsets = new HashMap<>(); + for (PartitionAssignment partition: newPartitions) { + SequenceOffsetType startOffset = currOffsets.computeIfAbsent( + partition.getPartitionId(), + k -> partition.getStartOffset() + ); + newPartitionStartOffsets.put(partition.getPartitionId(), startOffset); + newPartitionEndOffsets.put(partition.getPartitionId(), endOffsets.getOrDefault(partition.getPartitionId(), partition.getEndOffset())); + } + final Set exclusiveStartPartitions = computeExclusiveStartPartitionsForSequence(newPartitionStartOffsets); + final SequenceMetadata newSequence = new SequenceMetadata<>( + sequences.isEmpty() ? 0 : getLastSequenceMetadata().getSequenceId() + 1, + StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), sequences.isEmpty() ? 0 : getLastSequenceMetadata().getSequenceId() + 1), + newPartitionStartOffsets, + newPartitionEndOffsets, + false, + exclusiveStartPartitions, + getTaskLockType() + ); + + currOffsets.clear(); + currOffsets.putAll(newPartitionStartOffsets); + endOffsets.clear(); + endOffsets.putAll(newPartitionEndOffsets); + + addSequence(newSequence); + persistSequences(); + log.info("Created new sequence [%s] for partitions [%s] with start offsets [%s]", + newSequence.getSequenceName(), newPartitions, newPartitionStartOffsets); + } + + private void checkpointSequences() + { + try { + final SequenceMetadata latestSequence = getLastSequenceMetadata(); + if (!latestSequence.isCheckpointed()) { + final CheckPointDataSourceMetadataAction checkpointAciton = new CheckPointDataSourceMetadataAction( + getSupervisorId(), + ioConfig.getTaskGroupId(), + null, + createDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>( + stream, + latestSequence.getStartOffsets(), + latestSequence.getExclusiveStartPartitions() + ) + ) + ); + toolbox.getTaskActionClient().submit(checkpointAciton); + } + } + catch (Exception e) { + log.error(e, "Failed to checkpoint sequences."); + backgroundThreadException = e; + } + } + @VisibleForTesting public Response setEndOffsets( Map sequenceNumbers, From 5fe8a1de24a618c9cd43364ac27292facdb8e9d4 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 3 Sep 2025 10:59:16 +0530 Subject: [PATCH 02/46] Introduce updateConfig API --- .../seekablestream/PartitionAssignment.java | 96 ------------------- .../SeekableStreamIndexTaskRunner.java | 67 ++++++++----- ...uest.java => TaskConfigUpdateRequest.java} | 30 +++--- 3 files changed, 55 insertions(+), 138 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java rename indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/{PartitionReassignmentRequest.java => TaskConfigUpdateRequest.java} (56%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java deleted file mode 100644 index f8e6214749ef..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionAssignment.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.seekablestream; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.Objects; - -/** - * Represents a single partition assignment with its start and end offsets. - */ -public class PartitionAssignment -{ - private final PartitionIdType partitionId; - private final SequenceOffsetType startOffset; - private final SequenceOffsetType endOffset; - - @JsonCreator - public PartitionAssignment( - @JsonProperty("partitionId") PartitionIdType partitionId, - @JsonProperty("startOffset") SequenceOffsetType startOffset, - @JsonProperty("endOffset") SequenceOffsetType endOffset - ) - { - this.partitionId = partitionId; - this.startOffset = startOffset; - this.endOffset = endOffset; - } - - @JsonProperty - public PartitionIdType getPartitionId() - { - return partitionId; - } - - @JsonProperty - public SequenceOffsetType getStartOffset() - { - return startOffset; - } - - @JsonProperty - public SequenceOffsetType getEndOffset() - { - return endOffset; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - PartitionAssignment that = (PartitionAssignment) o; - return Objects.equals(partitionId, that.partitionId) && - Objects.equals(startOffset, that.startOffset) && - Objects.equals(endOffset, that.endOffset); - } - - @Override - public int hashCode() - { - return Objects.hash(partitionId, startOffset, endOffset); - } - - @Override - public String toString() - { - return "PartitionAssignment{" + - "partitionId=" + partitionId + - ", startOffset=" + startOffset + - ", endOffset=" + endOffset + - '}'; - } -} \ No newline at end of file diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 0d32b8876fd9..19dced0ee0c0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -161,7 +161,7 @@ public enum Status static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; - private final Map endOffsets; + private Map endOffsets; // lastReadOffsets are the last offsets that were read and processed. private final Map lastReadOffsets = new HashMap<>(); @@ -206,14 +206,14 @@ public enum Status protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); private final SeekableStreamIndexTask task; - private final SeekableStreamIndexTaskIOConfig ioConfig; + private SeekableStreamIndexTaskIOConfig ioConfig; private final SeekableStreamIndexTaskTuningConfig tuningConfig; private final InputRowSchema inputRowSchema; @Nullable private final InputFormat inputFormat; @Nullable private final InputRowParser parser; - private final String stream; + private String stream; private final Set publishingSequences = Sets.newConcurrentHashSet(); private final Set publishedSequences = Sets.newConcurrentHashSet(); @@ -1718,54 +1718,69 @@ public Response getUnparseableEvents( } @POST - @Path("/updateAssignments") + @Path("/updateConfig") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public void reassignPartitions(PartitionReassignmentRequest req) throws InterruptedException + public Response updateConfig(TaskConfigUpdateRequest req) throws InterruptedException { try { requestPause(); checkpointSequences(); - createNewSequenceForPartitions(req.getPartitionAssignments()); + + this.ioConfig = req.getIoConfig(); + this.stream = ioConfig.getStartSequenceNumbers().getStream(); + this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); + minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); + maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); + + createNewSequenceFromIoConfig(req.getIoConfig()); resume(); + return Response.ok().build(); } catch (Exception e) { - log.makeAlert(e, "Failed to reassign partitions."); + log.makeAlert(e, "Failed to update task config"); + return Response.serverError().entity(e.getMessage()).build(); } } - private void createNewSequenceForPartitions(List> newPartitions) + /** + * Creates new sequences for the ingestion process. It currently accepts the ioConfig given by the request as the correct offsets + * and ignores the offsets it may have stored in currOffsets and endOffsets. + */ + private void createNewSequenceFromIoConfig(SeekableStreamIndexTaskIOConfig ioConfig) throws IOException { - Map newPartitionStartOffsets = new HashMap<>(); - Map newPartitionEndOffsets = new HashMap<>(); - for (PartitionAssignment partition: newPartitions) { - SequenceOffsetType startOffset = currOffsets.computeIfAbsent( - partition.getPartitionId(), - k -> partition.getStartOffset() - ); - newPartitionStartOffsets.put(partition.getPartitionId(), startOffset); - newPartitionEndOffsets.put(partition.getPartitionId(), endOffsets.getOrDefault(partition.getPartitionId(), partition.getEndOffset())); - } - final Set exclusiveStartPartitions = computeExclusiveStartPartitionsForSequence(newPartitionStartOffsets); + Map partitionStartOffsets = ioConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap(); + Map partitionEndSequences = ioConfig.getEndSequenceNumbers() + .getPartitionSequenceNumberMap(); + + final Set exclusiveStartPartitions = computeExclusiveStartPartitionsForSequence( + partitionStartOffsets); final SequenceMetadata newSequence = new SequenceMetadata<>( sequences.isEmpty() ? 0 : getLastSequenceMetadata().getSequenceId() + 1, - StringUtils.format("%s_%d", ioConfig.getBaseSequenceName(), sequences.isEmpty() ? 0 : getLastSequenceMetadata().getSequenceId() + 1), - newPartitionStartOffsets, - newPartitionEndOffsets, + StringUtils.format( + "%s_%d", + ioConfig.getBaseSequenceName(), + sequences.isEmpty() ? 0 : getLastSequenceMetadata().getSequenceId() + 1 + ), + partitionStartOffsets, + partitionEndSequences, false, exclusiveStartPartitions, getTaskLockType() ); currOffsets.clear(); - currOffsets.putAll(newPartitionStartOffsets); + currOffsets.putAll(partitionStartOffsets); endOffsets.clear(); - endOffsets.putAll(newPartitionEndOffsets); + endOffsets.putAll(partitionEndSequences); addSequence(newSequence); persistSequences(); - log.info("Created new sequence [%s] for partitions [%s] with start offsets [%s]", - newSequence.getSequenceName(), newPartitions, newPartitionStartOffsets); + log.info( + "Created new sequence [%s] with start offsets [%s]", + newSequence.getSequenceName(), partitionStartOffsets + ); } private void checkpointSequences() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionReassignmentRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java similarity index 56% rename from indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionReassignmentRequest.java rename to indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java index 66c30d606d8c..8e31b9476659 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/PartitionReassignmentRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -22,29 +22,27 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; import java.util.Objects; /** - * Request object for reassigning partitions to a seekable stream index task. - * Contains a list of partition assignments, each with partition ID, start offset, and end offset. + * Request object for updating the configuration of a running {@link SeekableStreamIndexTask}. */ -public class PartitionReassignmentRequest +public class TaskConfigUpdateRequest { - private final List> partitionAssignments; + private final SeekableStreamIndexTaskIOConfig ioConfig; @JsonCreator - public PartitionReassignmentRequest( - @JsonProperty("partitionAssignments") List> partitionAssignments + public TaskConfigUpdateRequest( + @JsonProperty("ioConfig") SeekableStreamIndexTaskIOConfig ioConfig ) { - this.partitionAssignments = partitionAssignments; + this.ioConfig = ioConfig; } @JsonProperty - public List> getPartitionAssignments() + public SeekableStreamIndexTaskIOConfig getIoConfig() { - return partitionAssignments; + return ioConfig; } @Override @@ -56,21 +54,21 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - PartitionReassignmentRequest that = (PartitionReassignmentRequest) o; - return Objects.equals(partitionAssignments, that.partitionAssignments); + TaskConfigUpdateRequest that = (TaskConfigUpdateRequest) o; + return Objects.equals(ioConfig, that.ioConfig); } @Override public int hashCode() { - return Objects.hash(partitionAssignments); + return Objects.hash(ioConfig); } @Override public String toString() { - return "PartitionReassignmentRequest{" + - "partitionAssignments=" + partitionAssignments + + return "TaskConfigUpdateRequest{" + + "ioConfig=" + ioConfig + '}'; } -} \ No newline at end of file +} From 16960c08b777388ab9aad56bc3ff7a769757017d Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 4 Sep 2025 11:33:45 +0530 Subject: [PATCH 03/46] Update config request to just use generic objects --- .../seekablestream/SeekableStreamIndexTaskRunner.java | 10 +++++++--- .../seekablestream/TaskConfigUpdateRequest.java | 11 ++++++----- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 19dced0ee0c0..3d3ea104e7fe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1721,19 +1721,23 @@ public Response getUnparseableEvents( @Path("/updateConfig") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public Response updateConfig(TaskConfigUpdateRequest req) throws InterruptedException + public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context final HttpServletRequest req) throws InterruptedException { + authorizationCheck(req); try { requestPause(); checkpointSequences(); - this.ioConfig = req.getIoConfig(); + @SuppressWarnings("unchecked") + SeekableStreamIndexTaskIOConfig newIoConfig = + (SeekableStreamIndexTaskIOConfig) updateRequest.getIoConfig(); + this.ioConfig = newIoConfig; this.stream = ioConfig.getStartSequenceNumbers().getStream(); this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); - createNewSequenceFromIoConfig(req.getIoConfig()); + createNewSequenceFromIoConfig(newIoConfig); resume(); return Response.ok().build(); } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java index 8e31b9476659..b6862a06b5ad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -22,25 +22,26 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import javax.annotation.Nullable; import java.util.Objects; /** * Request object for updating the configuration of a running {@link SeekableStreamIndexTask}. */ -public class TaskConfigUpdateRequest +public class TaskConfigUpdateRequest { - private final SeekableStreamIndexTaskIOConfig ioConfig; + private final Object ioConfig; @JsonCreator public TaskConfigUpdateRequest( - @JsonProperty("ioConfig") SeekableStreamIndexTaskIOConfig ioConfig + @JsonProperty("ioConfig") @Nullable Object ioConfig ) { this.ioConfig = ioConfig; } @JsonProperty - public SeekableStreamIndexTaskIOConfig getIoConfig() + public Object getIoConfig() { return ioConfig; } @@ -54,7 +55,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - TaskConfigUpdateRequest that = (TaskConfigUpdateRequest) o; + TaskConfigUpdateRequest that = (TaskConfigUpdateRequest) o; return Objects.equals(ioConfig, that.ioConfig); } From 3fd66cbc5145b5492d60ba3e1ca2a7b6ac1d8c70 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 4 Sep 2025 12:16:33 +0530 Subject: [PATCH 04/46] Add configs to check whether perpetually running tasks should be used or not --- .../indexing/rabbitstream/RabbitStreamIndexTask.java | 3 ++- .../apache/druid/indexing/kafka/KafkaIndexTask.java | 4 +++- .../indexing/kafka/supervisor/KafkaSupervisor.java | 1 + .../kafka/supervisor/KafkaSupervisorSpec.java | 5 ++++- .../druid/indexing/kafka/KafkaIndexTaskTest.java | 1 + .../kafka/supervisor/KafkaSupervisorTest.java | 1 + .../druid/indexing/kinesis/KinesisIndexTask.java | 3 ++- .../seekablestream/SeekableStreamIndexTask.java | 11 ++++++++++- .../supervisor/SeekableStreamSupervisorSpec.java | 11 ++++++++++- .../SeekableStreamIndexTaskRunnerAuthTest.java | 2 +- .../supervisor/SeekableStreamSupervisorStateTest.java | 3 ++- .../test/java/org/apache/druid/cli/CliPeonTest.java | 2 +- 12 files changed, 38 insertions(+), 9 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java index 4aceebe37be7..b742fef389b9 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java @@ -67,7 +67,8 @@ public RabbitStreamIndexTask( tuningConfig, ioConfig, context, - getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE) + getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + null ); this.configMapper = configMapper; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index f19ac81a85b1..e8001ba5fc2d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -69,6 +69,7 @@ public KafkaIndexTask( @JsonProperty("tuningConfig") KafkaIndexTaskTuningConfig tuningConfig, @JsonProperty("ioConfig") KafkaIndexTaskIOConfig ioConfig, @JsonProperty("context") Map context, + @JsonProperty("isPerpetuallyRunning") @Nullable Boolean isPerpetuallyRunning, @JacksonInject ObjectMapper configMapper ) { @@ -80,7 +81,8 @@ public KafkaIndexTask( tuningConfig, ioConfig, context, - getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE) + getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + isPerpetuallyRunning ); this.configMapper = configMapper; 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 68bdb3fb4050..7d85980411b7 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 @@ -252,6 +252,7 @@ protected List context, @JsonProperty("suspended") Boolean suspended, + @JsonProperty("usePerpetuallyRunningTasks") @Nullable Boolean usePerpetuallyRunningTasks, @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, @@ -93,7 +94,8 @@ public KafkaSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig + supervisorStateManagerConfig, + usePerpetuallyRunningTasks ); } @@ -165,6 +167,7 @@ protected KafkaSupervisorSpec toggleSuspend(boolean suspend) getIoConfig(), getContext(), suspend, + usePerpetuallyRunningTasks(), taskStorage, taskMaster, indexerMetadataStorageCoordinator, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 9954de88c26f..826963a20ec3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2931,6 +2931,7 @@ private KafkaIndexTask createTask( tuningConfig, ioConfig, context, + null, OBJECT_MAPPER ); task.setPollRetryMs(POLL_RETRY_MS); 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 9b80d2db41a0..7df9bd5acd15 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 @@ -5732,6 +5732,7 @@ private KafkaIndexTask createKafkaIndexTask( Duration.standardHours(2).getStandardMinutes() ), Collections.emptyMap(), + null, OBJECT_MAPPER ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index a09cfcfa5baf..3b8e7438ad76 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -83,7 +83,8 @@ public KinesisIndexTask( tuningConfig, ioConfig, context, - getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE) + getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + null ); this.useListShards = useListShards; this.awsCredentialsConfig = awsCredentialsConfig; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 24f74b8da8b5..75aa507d640b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -71,6 +71,7 @@ public abstract class SeekableStreamIndexTask ioConfig, @Nullable final Map context, - @Nullable final String groupId + @Nullable final String groupId, + @Nullable final Boolean isPerpetuallyRunning ) { super( @@ -106,6 +108,7 @@ public SeekableStreamIndexTask( : LockGranularity.SEGMENT; this.lockTypeToUse = TaskLocks.determineLockTypeForAppend(getContext()); this.supervisorId = Preconditions.checkNotNull(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), "supervisorId"); + this.isPerpetuallyRunning = Configs.valueOrDefault(isPerpetuallyRunning, false); } protected static String getFormattedGroupId(String supervisorId, String type) @@ -153,6 +156,12 @@ public SeekableStreamIndexTaskIOConfig getI return ioConfig; } + @JsonProperty + public boolean isPerpetuallyRunning() + { + return isPerpetuallyRunning; + } + @Override public TaskStatus runTask(final TaskToolbox toolbox) { 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 967652673f7c..71a5d195c3a8 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 @@ -78,6 +78,7 @@ private static SeekableStreamSupervisorIngestionSpec checkIngestionSchema( protected final DruidMonitorSchedulerConfig monitorSchedulerConfig; private final boolean suspended; protected final SupervisorStateManagerConfig supervisorStateManagerConfig; + protected final boolean usePerpetuallyRunningTasks; /** * Base constructor for SeekableStreamSupervisors. @@ -97,7 +98,8 @@ public SeekableStreamSupervisorSpec( ServiceEmitter emitter, DruidMonitorSchedulerConfig monitorSchedulerConfig, RowIngestionMetersFactory rowIngestionMetersFactory, - SupervisorStateManagerConfig supervisorStateManagerConfig + SupervisorStateManagerConfig supervisorStateManagerConfig, + @Nullable Boolean usePerpetuallyRunningTasks ) { this.ingestionSchema = checkIngestionSchema(ingestionSchema); @@ -117,6 +119,7 @@ public SeekableStreamSupervisorSpec( this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.suspended = suspended != null ? suspended : false; this.supervisorStateManagerConfig = supervisorStateManagerConfig; + this.usePerpetuallyRunningTasks = Configs.valueOrDefault(usePerpetuallyRunningTasks, false); } @JsonProperty @@ -144,6 +147,12 @@ public SeekableStreamSupervisorIOConfig getIoConfig() return ingestionSchema.getIOConfig(); } + @JsonProperty + public boolean usePerpetuallyRunningTasks() + { + return usePerpetuallyRunningTasks; + } + @Nullable @JsonProperty public Map getContext() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index ec71cec89c18..7c319d8887aa 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -375,7 +375,7 @@ public TestSeekableStreamIndexTask( SeekableStreamIndexTaskIOConfig ioConfig ) { - super(id, null, null, dataSchema, tuningConfig, ioConfig, null, null); + super(id, null, null, dataSchema, tuningConfig, ioConfig, null, null, null); } @Override 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 54bf3c84bce6..6b82300e3ffb 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 @@ -2905,7 +2905,8 @@ public TestSeekableStreamIndexTask( tuningConfig, ioConfig, context, - groupId + groupId, + null ); this.streamingTaskRunner = streamingTaskRunner; } 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 4a6a3660c335..e6fdd8d772ec 100644 --- a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java +++ b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java @@ -266,7 +266,7 @@ private TestStreamingTask( ) { - super(id, supervisorId, taskResource, dataSchema, tuningConfig, ioConfig, context, groupId); + super(id, supervisorId, taskResource, dataSchema, tuningConfig, ioConfig, context, groupId, null); } @Override From b974ec813d0fc608f93c6fbe82dcfe687cd490f4 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 4 Sep 2025 19:29:16 +0530 Subject: [PATCH 05/46] Make partition assignment sequential --- .../kafka/supervisor/KafkaSupervisor.java | 26 ++++++++++++++++- .../SeekableStreamIndexTaskRunner.java | 28 ++++++++++++++++--- 2 files changed, 49 insertions(+), 5 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 7d85980411b7..ba45d7d89ee4 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 @@ -141,10 +141,34 @@ protected RecordSupplier setupReco protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) { Integer taskCount = spec.getIoConfig().getTaskCount(); + + if (spec.usePerpetuallyRunningTasks()) { + int taskGroupId = getRangeBasedTaskGroupId(partitionId, taskCount); + log.debug("Range-based assignment for partition [%s]: taskGroupId [%d]", partitionId, taskGroupId); + return taskGroupId; + } else { + if (partitionId.isMultiTopicPartition()) { + return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; + } else { + return partitionId.partition() % taskCount; + } + } + } + + /** + * Assigns partitions to task groups using range-based sequential assignment. + * This ensures that adjacent partitions are assigned to the same task group + */ + private int getRangeBasedTaskGroupId(KafkaTopicPartition partitionId, Integer taskCount) + { + int totalPartitions = partitionIds.size(); + int minPartitionsPerTaskGroup = totalPartitions / taskCount; + + if (partitionId.isMultiTopicPartition()) { return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; } else { - return partitionId.partition() % taskCount; + return partitionId.partition() / minPartitionsPerTaskGroup; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 3d3ea104e7fe..01890bce89a4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -251,6 +251,7 @@ public enum Status private volatile DateTime minMessageTime; private volatile DateTime maxMessageTime; private final ScheduledExecutorService rejectionPeriodUpdaterExec; + private volatile boolean isConfigChangeOngoing = false; public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, @@ -1600,6 +1601,15 @@ public Map getEndOffsets() return endOffsets; } + @GET + @Path("/config") + @Produces(MediaType.APPLICATION_JSON) + public SeekableStreamIndexTaskIOConfig getIOConfigHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req); + return ioConfig; + } + @POST @Path("/offsets/end") @Consumes(MediaType.APPLICATION_JSON) @@ -1725,7 +1735,8 @@ public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context fin { authorizationCheck(req); try { - requestPause(); + pause(); + isConfigChangeOngoing = true; checkpointSequences(); @SuppressWarnings("unchecked") @@ -1738,6 +1749,7 @@ public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context fin maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); createNewSequenceFromIoConfig(newIoConfig); + isConfigChangeOngoing = false; resume(); return Response.ok().build(); } catch (Exception e) { @@ -1834,7 +1846,11 @@ public Response setEndOffsets( .build(); } else { try { - pauseLock.lockInterruptibly(); + // Don't acquire a lock if a config change is ongoing, as the runner is already paused. + if (!isConfigChangeOngoing) + { + pauseLock.lockInterruptibly(); + } // Perform all sequence related checks before checking for isPaused() // and after acquiring pauseLock to correctly guard against duplicate requests Preconditions.checkState(sequenceNumbers.size() > 0, "No sequences found to set end sequences"); @@ -1858,7 +1874,9 @@ public Response setEndOffsets( || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resetNextCheckpointTime(); - resume(); + if (!isConfigChangeOngoing) { + resume(); + } return Response.ok(sequenceNumbers).build(); } else if (latestSequence.isCheckpointed()) { return Response.status(Response.Status.BAD_REQUEST) @@ -1937,7 +1955,9 @@ public Response setEndOffsets( } } - resume(); + if (!isConfigChangeOngoing) { + resume(); + } return Response.ok(sequenceNumbers).build(); } From 26379c546fa61818539a1e482f26f10a0808ee93 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 5 Sep 2025 13:38:29 +0530 Subject: [PATCH 06/46] Implement supervisor changes to handle perpetual tasks --- .../kafka/supervisor/KafkaSupervisor.java | 67 +++++ .../kinesis/supervisor/KinesisSupervisor.java | 10 + .../SeekableStreamIndexTaskClient.java | 7 + ...eekableStreamIndexTaskClientAsyncImpl.java | 24 ++ .../supervisor/SeekableStreamSupervisor.java | 235 +++++++++++++++++- 5 files changed, 340 insertions(+), 3 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index ba45d7d89ee4..793f482f1d17 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 @@ -250,6 +250,73 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ) + { + log.info("Creating updated task IO config for task group [%s]", existingTaskGroup.getId()); + Map startingSequences = new HashMap<>(); + Set exclusiveStartSequenceNumberPartitions = new HashSet<>(); + + for (KafkaTopicPartition partition : partitions) { + Long offset; + if (!latestCommittedOffsets.containsKey(partition)) { + log.warn("No committed offset found for partition [%s], using NOT_SET", partition); + offset = NOT_SET; + } else { + offset = latestCommittedOffsets.get(partition); + } + + startingSequences.put(partition, offset); + } + + SeekableStreamStartSequenceNumbers startSequenceNumbers = + new SeekableStreamStartSequenceNumbers<>( + spec.getIoConfig().getStream(), + startingSequences, + exclusiveStartSequenceNumberPartitions + ); + + // For end sequences, use NOT_SET to indicate open-ended reading + Map endingSequences = new HashMap<>(); + for (KafkaTopicPartition partition : partitions) { + endingSequences.put(partition, NOT_SET); + } + + SeekableStreamEndSequenceNumbers endSequenceNumbers = + new SeekableStreamEndSequenceNumbers<>( + spec.getIoConfig().getStream(), + endingSequences + ); + + log.info( + "Created updated IOConfig with starting sequences [%s] for partitions [%s]", + startingSequences, partitions + ); + + // Create the updated IOConfig + return new KafkaIndexTaskIOConfig( + existingTaskGroup.getId(), + existingTaskGroup.getBaseSequenceName(), + null, + null, + startSequenceNumbers, + endSequenceNumbers, + spec.getIoConfig().getConsumerProperties(), + spec.getIoConfig().getPollTimeout(), + true, + existingTaskGroup.getMinimumMessageTime(), + existingTaskGroup.getMaximumMessageTime(), + spec.getIoConfig().getInputFormat(), + spec.getIoConfig().getConfigOverrides(), + spec.getIoConfig().isMultiTopic(), + spec.getIoConfig().getTaskDuration().getStandardMinutes() + ); + } + @Override protected List> createIndexTasks( int replicas, 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 884476181fa6..cf22d06dee41 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 @@ -149,6 +149,16 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ) + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override protected List> createIndexTasks( int replicas, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 484ad15f7cd6..92e9ff2b0634 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -168,6 +168,13 @@ ListenableFuture registerNewVersionOfPendingSegmentAsync( PendingSegmentRecord pendingSegmentRecord ); + /** + * Update the configuration of a running task. + * + * Task-side is {@link SeekableStreamIndexTaskRunner#updateConfig}. + */ + ListenableFuture updateConfigAsync(String taskId, TaskConfigUpdateRequest updateRequest); + Class getPartitionType(); Class getSequenceType(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 6af2735e995b..447e9ca27245 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -210,6 +210,30 @@ public ListenableFuture registerNewVersionOfPendingSegmentAsync( .go(); } + @Override + public ListenableFuture updateConfigAsync(String taskId, TaskConfigUpdateRequest updateRequest) + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/updateConfig") + .jsonContent(jsonMapper, updateRequest); + + return makeRequest(taskId, requestBuilder) + .handler(IgnoreHttpResponseHandler.INSTANCE) + .onSuccess(r -> true) + .onHttpError(e -> { + log.warn("Task [%s] config update failed due to http request failure [%s].", taskId, e.getMessage()); + return Either.value(false); + }) + .onNotAvailable(e -> { + log.warn("Task [%s] config update failed because task is not available.", taskId); + return Either.value(false); + }) + .onClosed(e -> { + log.warn("Task [%s] config update failed because task is no longer running.", taskId); + return Either.value(false); + }) + .go(); + } + @Override public ListenableFuture setEndOffsetsAsync( final String id, 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 0a37f54cd04c..460ee71a5952 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 @@ -73,6 +73,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.TaskConfigUpdateRequest; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamException; @@ -200,9 +201,11 @@ public class TaskGroup // We don't include closed partitions in the starting offsets. However, we keep the full unfiltered map of // partitions, only used for generating the sequence name, to avoid ambiguity in sequence names if mulitple // task groups have nothing but closed partitions in their assignments. + final ImmutableMap unfilteredStartingSequencesForSequenceName; final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + final DateTime minimumMessageTime; final DateTime maximumMessageTime; final Set exclusiveStartSequenceNumberPartitions; @@ -212,6 +215,21 @@ public class TaskGroup boolean handoffEarly = false; // set by SupervisorManager.stopTaskGroupEarly + public int getId() + { + return groupId; + } + + public DateTime getMinimumMessageTime() + { + return minimumMessageTime; + } + + public DateTime getMaximumMessageTime() + { + return maximumMessageTime; + } + TaskGroup( int groupId, ImmutableMap startingSequences, @@ -564,9 +582,15 @@ private boolean changeTaskCount(int desiredActiveTaskCount) dataSource ); final Stopwatch scaleActionStopwatch = Stopwatch.createStarted(); - gracefulShutdownInternal(); - changeTaskCountInIOConfig(desiredActiveTaskCount); - clearAllocationInfo(); + + if (spec.usePerpetuallyRunningTasks()) { + return changeTaskCountForPerpetualTasks(desiredActiveTaskCount, scaleActionStopwatch); + } else { + gracefulShutdownInternal(); + changeTaskCountInIOConfig(desiredActiveTaskCount); + clearAllocationInfo(); + } + emitter.emit(ServiceMetricEvent.builder() .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) .setDimension(DruidMetrics.DATASOURCE, dataSource) @@ -584,6 +608,211 @@ private boolean changeTaskCount(int desiredActiveTaskCount) } } + private Map getLatestOffsetsFromMetadataStore() { + try { + DataSourceMetadata metadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(dataSource); + if (metadata instanceof SeekableStreamDataSourceMetadata) { + @SuppressWarnings("unchecked") + SeekableStreamDataSourceMetadata streamMetadata = + (SeekableStreamDataSourceMetadata) metadata; + return streamMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap(); + } + } catch (Exception e) { + log.warn(e, "Failed to retrieve latest offsets from metadata store, using current partition state"); + } + return Collections.emptyMap(); + } + + /** + * Handles task count changes for perpetual tasks using updateConfig instead of graceful shutdown. + * This approach pauses tasks, recalculates partition assignments, and sends config updates. + */ + private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount, Stopwatch scaleActionStopwatch) + throws InterruptedException, ExecutionException + { + log.info("Handling task count change for perpetual tasks from [%d] to [%d]", + activelyReadingTaskGroups.size(), desiredActiveTaskCount); + + pauseAllTasks(); + changeTaskCountInIOConfig(desiredActiveTaskCount); + Map> newPartitionGroups = calculateNewPartitionGroups(); + + boolean success = sendConfigUpdatesToTasks(newPartitionGroups); + + if (success) { + updatePartitionGroupsForPerpetualTasks(newPartitionGroups); + log.info("Successfully updated task configurations for perpetual tasks scaling"); + } else { + log.error("Failed to update task configurations for perpetual tasks"); + } + + return success; + } + + private void pauseAllTasks() throws InterruptedException, ExecutionException + { + log.info("Pausing all tasks for perpetual task scaling"); + List>> pauseFutures = new ArrayList<>(); + + for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { + for (String taskId : taskGroup.taskIds()) { + log.debug("Pausing task [%s]", taskId); + pauseFutures.add(taskClient.pauseAsync(taskId)); + } + } + + coalesceAndAwait(pauseFutures); + log.info("Successfully paused [%d] tasks", pauseFutures.size()); + } + + /** + * Calculates new partition groups using the existing getTaskGroupIdForPartition() logic. + */ + private Map> calculateNewPartitionGroups() + { + log.info("Calculating new partition groups using getTaskGroupIdForPartition() logic"); + Map> newPartitionGroups = new HashMap<>(); + + List allPartitions = new ArrayList<>(partitionIds); + + if (allPartitions.isEmpty()) { + log.warn("No partitions available for assignment"); + return newPartitionGroups; + } + + for (PartitionIdType partition : allPartitions) { + int taskGroupId = getTaskGroupIdForPartition(partition); + + if (taskGroupId >= 0 && taskGroupId < ioConfig.getTaskCount()) { + newPartitionGroups.computeIfAbsent(taskGroupId, k -> new HashSet<>()).add(partition); + } else { + log.warn("Invalid task group ID [%d] for partition [%s], skipping", taskGroupId, partition); + } + } + + log.info("Created [%d] new partition groups: %s", newPartitionGroups.size(), newPartitionGroups); + return newPartitionGroups; + } + + /** + * Sends configuration updates to tasks with new partition assignments. + * Also handles cleanup of obsolete task groups when scaling down. + */ + private boolean sendConfigUpdatesToTasks(Map> newPartitionGroups) + throws InterruptedException, ExecutionException + { + log.info("Sending configuration updates to tasks"); + List> updateFutures = new ArrayList<>(); + Map latestCommittedOffsets = getLatestOffsetsFromMetadataStore(); + for (Map.Entry> entry : newPartitionGroups.entrySet()) { + int taskGroupId = entry.getKey(); + Set partitions = entry.getValue(); + + TaskGroup existingTaskGroup = activelyReadingTaskGroups.get(taskGroupId); + if (existingTaskGroup != null) { + for (String taskId : existingTaskGroup.taskIds()) { + SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig(partitions, existingTaskGroup, latestCommittedOffsets); + TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); + + log.debug("Updating config for task [%s] with partitions [%s]", taskId, partitions); + updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); + } + } + } + + + if (updateFutures.isEmpty()) { + log.info("No configuration updates needed"); + return true; + } + + List> results = coalesceAndAwait(updateFutures); + boolean allSucceeded = results.stream().allMatch(result -> { + if (result.isValue()) { + Boolean value = result.valueOrThrow(); + return value != null && value; + } + return false; + }); + + if (allSucceeded) { + log.info("Successfully sent configuration updates to [%d] tasks", updateFutures.size()); + } else { + log.error("Some configuration updates failed"); + } + handleObsoleteTaskGroups(newPartitionGroups.keySet()); + + return allSucceeded; + } + + /** + * Handles obsolete task groups when scaling down. + * Pauses tasks in task groups that are no longer needed and removes them from activelyReadingTaskGroups. + */ + private void handleObsoleteTaskGroups(Set newTaskGroupIds) + { + Set currentTaskGroupIds = new HashSet<>(activelyReadingTaskGroups.keySet()); + Set obsoleteTaskGroupIds = new HashSet<>(currentTaskGroupIds); + obsoleteTaskGroupIds.removeAll(newTaskGroupIds); + + if (obsoleteTaskGroupIds.isEmpty()) { + log.debug("No obsolete task groups to clean up"); + return; + } + + log.info("Handling obsolete task groups during scaling down: %s", obsoleteTaskGroupIds); + + for (Integer obsoleteTaskGroupId : obsoleteTaskGroupIds) { + TaskGroup obsoleteTaskGroup = activelyReadingTaskGroups.get(obsoleteTaskGroupId); + if (obsoleteTaskGroup != null) { + log.info("Pausing tasks in obsolete task group [%d]: %s", obsoleteTaskGroupId, obsoleteTaskGroup.taskIds()); + + // Pause all tasks in the obsolete task group + for (String taskId : obsoleteTaskGroup.taskIds()) { + try { + taskClient.pauseAsync(taskId); + log.info("Requested pause for task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); + } catch (Exception e) { + log.error(e, "Failed to pause task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); + } + } + + // Remove the task group from activelyReadingTaskGroups + // The supervisor's normal run cycle will handle shutdown of these paused tasks + activelyReadingTaskGroups.remove(obsoleteTaskGroupId); + log.info("Removed obsolete task group [%d] from activelyReadingTaskGroups", obsoleteTaskGroupId); + } + } + } + + /** + * Updates the partition groups mapping for perpetual tasks without clearing other allocation info. + */ + private void updatePartitionGroupsForPerpetualTasks(Map> newPartitionGroups) + { + log.info("Updating partition groups mapping for perpetual tasks"); + + // Update the partition groups mapping + partitionGroups.clear(); + partitionGroups.putAll(newPartitionGroups); + + // Update partition offsets for new partitions if needed + for (PartitionIdType partition : partitionIds) { + partitionOffsets.putIfAbsent(partition, getNotSetMarker()); + } + + log.info("Updated partition groups: %s", partitionGroups); + } + + /** + * Creates an updated IOConfig for a task with new partition assignments. + */ + protected abstract SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ); + private void changeTaskCountInIOConfig(int desiredActiveTaskCount) { ioConfig.setTaskCount(desiredActiveTaskCount); From 27dddae5fcf5cf47be9fdf7c9fac5e3659599110 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 5 Sep 2025 15:00:40 +0530 Subject: [PATCH 07/46] Fix compilation and checkstyle issues --- .../embedded/indexing/IngestionSmokeTest.java | 2 +- .../indexing/KafkaClusterMetricsTest.java | 2 +- .../indexing/KafkaDataFormatsTest.java | 4 +- .../embedded/msq/BaseRealtimeQueryTest.java | 1 + .../embedded/server/OverlordClientTest.java | 2 +- .../indexing/kafka/KafkaSamplerSpecTest.java | 6 +++ .../simulate/EmbeddedKafkaSupervisorTest.java | 2 +- .../supervisor/KafkaSupervisorSpecTest.java | 2 + .../kafka/supervisor/KafkaSupervisorTest.java | 4 ++ .../SeekableStreamIndexTaskRunner.java | 11 +++--- .../supervisor/SeekableStreamSupervisor.java | 39 +++++++++++-------- .../SeekableStreamSupervisorSpecTest.java | 21 +++++++++- .../SeekableStreamSupervisorStateTest.java | 31 +++++++++++++++ 13 files changed, 98 insertions(+), 29 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java index af9c3d21c8e8..7063a2cd8d5f 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/IngestionSmokeTest.java @@ -337,7 +337,7 @@ private KafkaSupervisorSpec createKafkaSupervisor(String topic) true, null, null, null, null, null, null, null, null ), - null, null, null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null, null, null ); } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 2cf03e99ce96..9a9bf55ec8b5 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -438,7 +438,7 @@ private KafkaSupervisorSpec createKafkaSupervisor( Period.millis(taskCompletionTimeoutMillis), null, null, null, null, null, null, null ), - null, null, null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null, null, null ); } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java index c8fff7347d5c..d0c442b9b6c7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaDataFormatsTest.java @@ -604,7 +604,7 @@ private KafkaSupervisorSpec createKafkaSupervisorWithParser(String supervisorId, true, null, null, null, null, null, null, null, null ), - null, null, null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null, null, null ); } @@ -632,7 +632,7 @@ private KafkaSupervisorSpec createKafkaSupervisor(String supervisorId, String to true, null, null, null, null, null, null, null, null ), - null, null, null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null, null, null ); } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java index 0d82c6f7850e..dae679f69e4e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/msq/BaseRealtimeQueryTest.java @@ -192,6 +192,7 @@ private KafkaSupervisorSpec createKafkaSupervisor() null, null, null, + null, null ); } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/OverlordClientTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/OverlordClientTest.java index 36de4c072a7f..e778c8a17f6e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/OverlordClientTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/OverlordClientTest.java @@ -227,7 +227,7 @@ public void test_postSupervisor_fails_ifRequiredExtensionIsNotLoaded() null, null, null, null, null, null, null, null, null, null, null, null, null, null ), Map.of(), - null, null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null, null ); final Exception exception = Assertions.assertThrows( diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 5e21fd0aa7f8..4af80a446c64 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -180,6 +180,7 @@ public void testSample() null, null, null, + null, null ); @@ -236,6 +237,7 @@ public void testSampleWithTopicPattern() null, null, null, + null, null ); @@ -301,6 +303,7 @@ public void testSampleKafkaInputFormat() null, null, null, + null, null ); @@ -408,6 +411,7 @@ public void testWithInputRowParser() throws IOException null, null, null, + null, null ); @@ -595,6 +599,7 @@ public void testInvalidKafkaConfig() null, null, null, + null, null ); @@ -654,6 +659,7 @@ public void testGetInputSourceResources() null, null, null, + null, null ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedKafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedKafkaSupervisorTest.java index 058f3ffa7367..4381b9e92b8b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedKafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/simulate/EmbeddedKafkaSupervisorTest.java @@ -167,7 +167,7 @@ private KafkaSupervisorSpec createKafkaSupervisor(String supervisorId, String to true, null, null, null, null, null, null, null, null ), - null, null, null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null, null, null ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java index d7bb9acfb7f6..b72330be7ff7 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecTest.java @@ -719,6 +719,7 @@ public void test_validateSpecUpdateTo() null, null, null, + null, null ); sourceSpec.validateSpecUpdateTo(validDestSpec); @@ -764,6 +765,7 @@ private KafkaSupervisorSpec getSpec(String topic, String topicPattern) null, null, null, + null, null ); } 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 7df9bd5acd15..295656f9e51b 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 @@ -374,6 +374,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, false, + null, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -5409,6 +5410,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, + null, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -5527,6 +5529,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, + null, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -5619,6 +5622,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, + null, taskStorage, taskMaster, indexerMetadataStorageCoordinator, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 01890bce89a4..57f8b80b8a0c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1731,7 +1731,8 @@ public Response getUnparseableEvents( @Path("/updateConfig") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context final HttpServletRequest req) throws InterruptedException + public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context final HttpServletRequest req) + throws InterruptedException { authorizationCheck(req); try { @@ -1740,7 +1741,7 @@ public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context fin checkpointSequences(); @SuppressWarnings("unchecked") - SeekableStreamIndexTaskIOConfig newIoConfig = + SeekableStreamIndexTaskIOConfig newIoConfig = (SeekableStreamIndexTaskIOConfig) updateRequest.getIoConfig(); this.ioConfig = newIoConfig; this.stream = ioConfig.getStartSequenceNumbers().getStream(); @@ -1752,7 +1753,8 @@ public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context fin isConfigChangeOngoing = false; resume(); return Response.ok().build(); - } catch (Exception e) { + } + catch (Exception e) { log.makeAlert(e, "Failed to update task config"); return Response.serverError().entity(e.getMessage()).build(); } @@ -1847,8 +1849,7 @@ public Response setEndOffsets( } else { try { // Don't acquire a lock if a config change is ongoing, as the runner is already paused. - if (!isConfigChangeOngoing) - { + if (!isConfigChangeOngoing) { pauseLock.lockInterruptibly(); } // Perform all sequence related checks before checking for isPaused() 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 460ee71a5952..8fee67b06bf9 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 @@ -608,7 +608,8 @@ private boolean changeTaskCount(int desiredActiveTaskCount) } } - private Map getLatestOffsetsFromMetadataStore() { + private Map getLatestOffsetsFromMetadataStore() + { try { DataSourceMetadata metadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(dataSource); if (metadata instanceof SeekableStreamDataSourceMetadata) { @@ -617,7 +618,8 @@ private Map getLatestOffsetsFromMetadataSto (SeekableStreamDataSourceMetadata) metadata; return streamMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap(); } - } catch (Exception e) { + } + catch (Exception e) { log.warn(e, "Failed to retrieve latest offsets from metadata store, using current partition state"); } return Collections.emptyMap(); @@ -672,24 +674,24 @@ private Map> calculateNewPartitionGroups() { log.info("Calculating new partition groups using getTaskGroupIdForPartition() logic"); Map> newPartitionGroups = new HashMap<>(); - + List allPartitions = new ArrayList<>(partitionIds); - + if (allPartitions.isEmpty()) { log.warn("No partitions available for assignment"); return newPartitionGroups; } - + for (PartitionIdType partition : allPartitions) { int taskGroupId = getTaskGroupIdForPartition(partition); - + if (taskGroupId >= 0 && taskGroupId < ioConfig.getTaskCount()) { newPartitionGroups.computeIfAbsent(taskGroupId, k -> new HashSet<>()).add(partition); } else { log.warn("Invalid task group ID [%d] for partition [%s], skipping", taskGroupId, partition); } } - + log.info("Created [%d] new partition groups: %s", newPartitionGroups.size(), newPartitionGroups); return newPartitionGroups; } @@ -707,13 +709,17 @@ private boolean sendConfigUpdatesToTasks(Map> newP for (Map.Entry> entry : newPartitionGroups.entrySet()) { int taskGroupId = entry.getKey(); Set partitions = entry.getValue(); - + TaskGroup existingTaskGroup = activelyReadingTaskGroups.get(taskGroupId); if (existingTaskGroup != null) { for (String taskId : existingTaskGroup.taskIds()) { - SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig(partitions, existingTaskGroup, latestCommittedOffsets); + SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig( + partitions, + existingTaskGroup, + latestCommittedOffsets + ); TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); - + log.debug("Updating config for task [%s] with partitions [%s]", taskId, partitions); updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); } @@ -754,29 +760,30 @@ private void handleObsoleteTaskGroups(Set newTaskGroupIds) Set currentTaskGroupIds = new HashSet<>(activelyReadingTaskGroups.keySet()); Set obsoleteTaskGroupIds = new HashSet<>(currentTaskGroupIds); obsoleteTaskGroupIds.removeAll(newTaskGroupIds); - + if (obsoleteTaskGroupIds.isEmpty()) { log.debug("No obsolete task groups to clean up"); return; } - + log.info("Handling obsolete task groups during scaling down: %s", obsoleteTaskGroupIds); - + for (Integer obsoleteTaskGroupId : obsoleteTaskGroupIds) { TaskGroup obsoleteTaskGroup = activelyReadingTaskGroups.get(obsoleteTaskGroupId); if (obsoleteTaskGroup != null) { log.info("Pausing tasks in obsolete task group [%d]: %s", obsoleteTaskGroupId, obsoleteTaskGroup.taskIds()); - + // Pause all tasks in the obsolete task group for (String taskId : obsoleteTaskGroup.taskIds()) { try { taskClient.pauseAsync(taskId); log.info("Requested pause for task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); - } catch (Exception e) { + } + catch (Exception e) { log.error(e, "Failed to pause task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); } } - + // Remove the task group from activelyReadingTaskGroups // The supervisor's normal run cycle will handle shutdown of these paused tasks activelyReadingTaskGroups.remove(obsoleteTaskGroupId); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index f0ba8966626e..a5f66e794ff8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -365,6 +365,17 @@ public LagStats computeLagStats() return new LagStats(0, 0, 0); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ) + { + // dummy implementation + return null; + } + @Override public int getPartitionCount() { @@ -423,7 +434,8 @@ public TestSeekableStreamSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig + supervisorStateManagerConfig, + null ); this.supervisor = supervisor; @@ -746,6 +758,7 @@ public void testSeekableStreamSupervisorSpecWithScaleOut() throws InterruptedExc EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -912,6 +925,7 @@ public void testSeekableStreamSupervisorSpecWithScaleOutSmallPartitionNumber() t EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -958,6 +972,7 @@ public void testSeekableStreamSupervisorSpecWithScaleIn() throws InterruptedExce EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1008,6 +1023,7 @@ public void testSeekableStreamSupervisorSpecWithScaleInThresholdGreaterThanParti EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1213,7 +1229,8 @@ public void testEnablingIdleBeviourPerSupervisorWithOverlordConfigEnabled() null, null, null, - supervisorStateManagerConfig + supervisorStateManagerConfig, + null ) { @Override 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 6b82300e3ffb..f7968ae07e1d 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 @@ -2948,6 +2948,17 @@ private BaseTestSeekableStreamSupervisor() ); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ) + { + // dummy implementation + return null; + } + @Override protected String baseTaskName() { @@ -3163,6 +3174,16 @@ public LagStats computeLagStats() return new LagStats(0, 0, 0); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ) + { + return null; + } + @Override protected Map getLatestSequencesFromStream() { @@ -3200,6 +3221,16 @@ private class TestEmittingTestSeekableStreamSupervisor extends BaseTestSeekableS this.partitionsTimeLag = partitionsTimeLag; } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ) + { + return null; + } + @Nullable @Override protected Map getPartitionRecordLag() From aa6732216b2934042012c715ee27fe645486ea62 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 5 Sep 2025 15:23:50 +0530 Subject: [PATCH 08/46] Fix kinesis compilation failures --- .../indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java | 2 +- .../indexing/kinesis/supervisor/KinesisSupervisorSpec.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java index ce36c0b7dc0c..40b460af45af 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java @@ -90,7 +90,7 @@ private KafkaSupervisorSpec build() null, false, // Jackson injected params, not needed while posting a supervisor to the Overlord - null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null ); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 8e6615716809..dea6fa3dbfa4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -94,7 +94,8 @@ public KinesisSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig + supervisorStateManagerConfig, + null ); this.awsCredentialsConfig = awsCredentialsConfig; } From a57d3ebc525439e375fa4cc3cd8f871ce92020cd Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 5 Sep 2025 15:37:58 +0530 Subject: [PATCH 09/46] Working compilation --- .../supervisor/RabbitStreamSupervisor.java | 10 ++++++++++ .../supervisor/RabbitStreamSupervisorSpec.java | 3 ++- 2 files changed, 12 insertions(+), 1 deletion(-) 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 2ced013ef5bc..eba5edfb71a1 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 @@ -402,6 +402,16 @@ protected Map getLatestSequencesFromStream() return latestSequenceFromStream != null ? latestSequenceFromStream : new HashMap<>(); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets + ) + { + throw new UnsupportedOperationException("not implemented"); + } + @Override protected String baseTaskName() { diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java index 4a445f6f1c11..c962782d7d9c 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java @@ -82,7 +82,8 @@ public RabbitStreamSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig); + supervisorStateManagerConfig, + null); } @Override From a690e179102540335fa8f5a4874fb5bc60b856ad Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 5 Sep 2025 22:11:19 +0530 Subject: [PATCH 10/46] Add embedded tests --- .../KafkaSupervisorSpecBuilder.java | 10 +- .../LagBasedAutoScalerConfigBuilder.java | 166 ++++++++++++++++++ 2 files changed, 175 insertions(+), 1 deletion(-) create mode 100644 extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java index 40b460af45af..513446b1c27b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java @@ -30,6 +30,7 @@ public class KafkaSupervisorSpecBuilder { private String id; + private Boolean usePerpetuallyRunningTasks; private final DataSchema.Builder dataSchema = new DataSchema.Builder(); private final KafkaIOConfigBuilder ioConfig = new KafkaIOConfigBuilder(); private final KafkaTuningConfigBuilder tuningConfig = new KafkaTuningConfigBuilder(); @@ -57,6 +58,12 @@ public KafkaSupervisorSpecBuilder withId(String id) this.id = id; return this; } + + public KafkaSupervisorSpecBuilder withUsePerpetuallyRunningTasks(Boolean usePerpetuallyRunningTasks) + { + this.usePerpetuallyRunningTasks = usePerpetuallyRunningTasks; + return this; + } /** * Builds a new {@link KafkaSupervisorSpec} with the specified parameters. @@ -89,8 +96,9 @@ private KafkaSupervisorSpec build() ioConfig.build(), null, false, + usePerpetuallyRunningTasks, // Jackson injected params, not needed while posting a supervisor to the Overlord - null, null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null ); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java new file mode 100644 index 000000000000..fb10a4ad89e6 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java @@ -0,0 +1,166 @@ +/* + * 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.indexing.overlord.supervisor.autoscaler.AggregateFunction; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.LagBasedAutoScalerConfig; + +import javax.annotation.Nullable; + +/** + * Builder for {@link LagBasedAutoScalerConfig} used in tests. + */ +public class LagBasedAutoScalerConfigBuilder +{ + private long lagCollectionIntervalMillis; + private long lagCollectionRangeMillis; + private long scaleActionStartDelayMillis; + private long scaleActionPeriodMillis; + private long scaleOutThreshold; + private long scaleInThreshold; + private double triggerScaleOutFractionThreshold; + private double triggerScaleInFractionThreshold; + private int taskCountMax; + private int taskCountMin; + private Integer taskCountStart; + private int scaleInStep; + private int scaleOutStep; + private boolean enableTaskAutoScaler; + private long minTriggerScaleActionFrequencyMillis; + private AggregateFunction lagAggregate; + + public LagBasedAutoScalerConfigBuilder withLagCollectionIntervalMillis(long lagCollectionIntervalMillis) + { + this.lagCollectionIntervalMillis = lagCollectionIntervalMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withLagCollectionRangeMillis(long lagCollectionRangeMillis) + { + this.lagCollectionRangeMillis = lagCollectionRangeMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleActionStartDelayMillis(long scaleActionStartDelayMillis) + { + this.scaleActionStartDelayMillis = scaleActionStartDelayMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleActionPeriodMillis(long scaleActionPeriodMillis) + { + this.scaleActionPeriodMillis = scaleActionPeriodMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleOutThreshold(long scaleOutThreshold) + { + this.scaleOutThreshold = scaleOutThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleInThreshold(long scaleInThreshold) + { + this.scaleInThreshold = scaleInThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTriggerScaleOutFractionThreshold(double triggerScaleOutFractionThreshold) + { + this.triggerScaleOutFractionThreshold = triggerScaleOutFractionThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTriggerScaleInFractionThreshold(double triggerScaleInFractionThreshold) + { + this.triggerScaleInFractionThreshold = triggerScaleInFractionThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTaskCountMax(int taskCountMax) + { + this.taskCountMax = taskCountMax; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTaskCountMin(int taskCountMin) + { + this.taskCountMin = taskCountMin; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTaskCountStart(@Nullable Integer taskCountStart) + { + this.taskCountStart = taskCountStart; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleInStep(int scaleInStep) + { + this.scaleInStep = scaleInStep; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleOutStep(int scaleOutStep) + { + this.scaleOutStep = scaleOutStep; + return this; + } + + public LagBasedAutoScalerConfigBuilder withEnableTaskAutoScaler(boolean enableTaskAutoScaler) + { + this.enableTaskAutoScaler = enableTaskAutoScaler; + return this; + } + + public LagBasedAutoScalerConfigBuilder withMinTriggerScaleActionFrequencyMillis(long minTriggerScaleActionFrequencyMillis) + { + this.minTriggerScaleActionFrequencyMillis = minTriggerScaleActionFrequencyMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withLagAggregate(AggregateFunction lagAggregate) + { + this.lagAggregate = lagAggregate; + return this; + } + + public LagBasedAutoScalerConfig build() + { + return new LagBasedAutoScalerConfig( + lagCollectionIntervalMillis, + lagCollectionRangeMillis, + scaleActionStartDelayMillis, + scaleActionPeriodMillis, + scaleOutThreshold, + scaleInThreshold, + triggerScaleOutFractionThreshold, + triggerScaleInFractionThreshold, + taskCountMax, + taskCountStart, + taskCountMin, + scaleInStep, + scaleOutStep, + enableTaskAutoScaler, + minTriggerScaleActionFrequencyMillis, + lagAggregate + ); + } +} From a85c85e22bc40e8b970a1733f15a95da60bb584d Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Sat, 6 Sep 2025 00:56:49 +0530 Subject: [PATCH 11/46] Get embedded tests working --- .../druid/indexing/kafka/supervisor/KafkaSupervisor.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 793f482f1d17..0ea0053988cb 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 @@ -76,6 +76,8 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; +import static java.lang.Math.max; + /** * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a * {@link KafkaSupervisorSpec} which includes the Kafka topic and configuration as well as an ingestion spec which will @@ -168,7 +170,7 @@ private int getRangeBasedTaskGroupId(KafkaTopicPartition partitionId, Integer ta if (partitionId.isMultiTopicPartition()) { return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; } else { - return partitionId.partition() / minPartitionsPerTaskGroup; + return max(taskCount - 1, partitionId.partition() / minPartitionsPerTaskGroup); } } From 7060e63b274381ee092f17ed0d2cfd76e09adc68 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Mon, 8 Sep 2025 12:43:43 +0530 Subject: [PATCH 12/46] Push embedded tests --- .../indexing/KafkaClusterMetricsTest.java | 93 ++++++++++++++++++- 1 file changed, 89 insertions(+), 4 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 5a977b861e81..3c4d28cff80f 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -27,7 +27,10 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; import org.apache.druid.indexing.kafka.simulate.KafkaResource; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpecBuilder; +import org.apache.druid.indexing.kafka.supervisor.LagBasedAutoScalerConfigBuilder; import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.rpc.indexing.OverlordClient; @@ -70,6 +73,7 @@ public class KafkaClusterMetricsTest extends EmbeddedClusterTestBase private final EmbeddedOverlord overlord = new EmbeddedOverlord(); private final EmbeddedHistorical historical = new EmbeddedHistorical(); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; @Override @@ -142,7 +146,8 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( supervisorId, taskCount, - maxRowsPerSegment + maxRowsPerSegment, + null ); Assertions.assertEquals( @@ -175,6 +180,69 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); } + @Test + @Timeout(60) + public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() + { + final int maxRowsPerSegment = 1000; + final int expectedSegmentsHandedOff = 30; + + final int taskCount = 2; + + // Submit and start a supervisor + final String supervisorId = dataSource + "_supe"; + AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() + .withLagCollectionIntervalMillis(100) + .withLagCollectionRangeMillis(100) + .withEnableTaskAutoScaler(true) + .withScaleActionPeriodMillis(3000) + .withScaleActionStartDelayMillis(1000) + .withScaleOutThreshold(0) + .withScaleInThreshold(10000) + .withTriggerScaleOutFractionThreshold(0.001) + .withTriggerScaleInFractionThreshold(0.1) + .withTaskCountMax(10) + .withTaskCountMin(taskCount) + .withScaleOutStep(1) + .withScaleInStep(0) + .withMinTriggerScaleActionFrequencyMillis(100) + .build(); + + final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( + supervisorId, + taskCount, + maxRowsPerSegment, + autoScalerConfig, + true + ); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + indexer.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("ingest/handoff/count") + .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)), + agg -> agg.hasSumAtLeast(expectedSegmentsHandedOff) + ); + + final int numSegments = Integer.parseInt( + cluster.runSql("SELECT COUNT(*) FROM sys.segments WHERE datasource = '%s'", dataSource) + ); + Assertions.assertTrue(numSegments >= expectedSegmentsHandedOff); + + final int numRows = Integer.parseInt( + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + Assertions.assertTrue(numRows >= expectedSegmentsHandedOff * maxRowsPerSegment); + + verifyIngestedMetricCountMatchesEmittedCount("jvm/pool/committed", coordinator); + verifyIngestedMetricCountMatchesEmittedCount("coordinator/time", coordinator); + + cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); + } + @Test @Timeout(120) public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkipKillOfUnusedSegments() @@ -189,7 +257,8 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( supervisorId, taskCount, - maxRowsPerSegment + maxRowsPerSegment, + null ); cluster.callApi().postSupervisor(kafkaSupervisorSpec); @@ -306,7 +375,8 @@ public void test_ingestClusterMetrics_compactionSkipsLockedIntervals() final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( supervisorId, taskCount, - maxRowsPerSegment + maxRowsPerSegment, + null ); cluster.callApi().postSupervisor(kafkaSupervisorSpec); @@ -385,7 +455,19 @@ private void verifyIngestedMetricCountMatchesEmittedCount(String metricName, Emb private KafkaSupervisorSpec createKafkaSupervisor( String supervisorId, int taskCount, - int maxRowsPerSegment + int maxRowsPerSegment, + AutoScalerConfig autoScalerConfig + ) + { + return createKafkaSupervisor(supervisorId, taskCount, maxRowsPerSegment, autoScalerConfig, false); + } + + private KafkaSupervisorSpec createKafkaSupervisor( + String supervisorId, + int taskCount, + int maxRowsPerSegment, + AutoScalerConfig autoScalerConfig, + boolean usePerpetuallyRunningTasks ) { return MoreResources.Supervisor.KAFKA_JSON @@ -396,8 +478,11 @@ private KafkaSupervisorSpec createKafkaSupervisor( ioConfig -> ioConfig .withConsumerProperties(kafkaServer.consumerProperties()) .withTaskCount(taskCount) + .withAutoScalerConfig(autoScalerConfig) + .withTaskDuration(Period.hours(1)) ) .withId(supervisorId) + .withUsePerpetuallyRunningTasks(usePerpetuallyRunningTasks) .build(dataSource, TOPIC); } } From 870ef5b51e4e3b396af015963a75e9068c978daf Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Mon, 8 Sep 2025 13:21:13 +0530 Subject: [PATCH 13/46] Perform parsing in task runner --- .../SeekableStreamIndexTaskRunner.java | 14 ++++++++++---- .../seekablestream/TaskConfigUpdateRequest.java | 2 ++ 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 57f8b80b8a0c..ff4ca960dd71 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -117,6 +117,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -1731,7 +1732,7 @@ public Response getUnparseableEvents( @Path("/updateConfig") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context final HttpServletRequest req) + public Response updateConfig(TaskConfigUpdateRequest request, @Context final HttpServletRequest req) throws InterruptedException { authorizationCheck(req); @@ -1740,9 +1741,13 @@ public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context fin isConfigChangeOngoing = true; checkpointSequences(); - @SuppressWarnings("unchecked") - SeekableStreamIndexTaskIOConfig newIoConfig = - (SeekableStreamIndexTaskIOConfig) updateRequest.getIoConfig(); + SeekableStreamIndexTaskIOConfig newIoConfig; + if (request.getIoConfig() instanceof LinkedHashMap) { + newIoConfig =(SeekableStreamIndexTaskIOConfig) + toolbox.getJsonMapper().convertValue(request.getIoConfig(), SeekableStreamIndexTaskIOConfig.class); + } else { + newIoConfig = (SeekableStreamIndexTaskIOConfig) request.getIoConfig(); + } this.ioConfig = newIoConfig; this.stream = ioConfig.getStartSequenceNumbers().getStream(); this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); @@ -1751,6 +1756,7 @@ public Response updateConfig(TaskConfigUpdateRequest updateRequest, @Context fin createNewSequenceFromIoConfig(newIoConfig); isConfigChangeOngoing = false; + log.info("Config updated to [%s]", newIoConfig); resume(); return Response.ok().build(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java index b6862a06b5ad..42029da70e19 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -21,8 +21,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import javax.annotation.Nullable; +import java.util.LinkedHashMap; import java.util.Objects; /** From 7b5524b759196007d752dab7e7190d85bb027189 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Mon, 8 Sep 2025 13:53:57 +0530 Subject: [PATCH 14/46] Emit events when config is updated --- .../seekablestream/SeekableStreamIndexTaskRunner.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index ff4ca960dd71..555af51bcbed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -78,7 +78,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -1737,6 +1739,7 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt { authorizationCheck(req); try { + log.info("Attempting to update config to [%s]", request.getIoConfig()); pause(); isConfigChangeOngoing = true; checkpointSequences(); @@ -1756,7 +1759,12 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt createNewSequenceFromIoConfig(newIoConfig); isConfigChangeOngoing = false; - log.info("Config updated to [%s]", newIoConfig); + log.info("Config updated to [%s]", this.ioConfig); + toolbox.getEmitter().emit(ServiceMetricEvent.builder() + .setDimension(DruidMetrics.TASK_ID, task.getId()) + .setDimension(DruidMetrics.TASK_TYPE, task.getType()) + .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) + .build("task/config/update/success", String.valueOf(1))); resume(); return Response.ok().build(); } From 77193e06902533e083e7bd8fa2ec72c5e0f61bf6 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 10 Sep 2025 11:33:51 +0530 Subject: [PATCH 15/46] WIP: Attempting to fix failing latch --- .../indexing/KafkaClusterMetricsTest.java | 10 +-- .../supervisor/RabbitStreamSupervisor.java | 3 +- .../kafka/supervisor/KafkaSupervisor.java | 17 ++--- .../kinesis/supervisor/KinesisSupervisor.java | 3 +- .../SeekableStreamIndexTaskClient.java | 2 + ...eekableStreamIndexTaskClientAsyncImpl.java | 45 ++++++++++++ .../SeekableStreamIndexTaskRunner.java | 70 +++++++++++++------ .../TaskConfigUpdateRequest.java | 6 +- .../supervisor/SeekableStreamSupervisor.java | 67 ++++++++++++++---- .../SeekableStreamSupervisorSpecTest.java | 3 +- .../SeekableStreamSupervisorStateTest.java | 9 ++- 11 files changed, 176 insertions(+), 59 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 3c4d28cff80f..6cfc17656d4e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -185,9 +185,9 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() { final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 30; + final int expectedSegmentsHandedOff = 50; - final int taskCount = 2; + final int taskCount = 1; // Submit and start a supervisor final String supervisorId = dataSource + "_supe"; @@ -195,7 +195,7 @@ public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues .withLagCollectionIntervalMillis(100) .withLagCollectionRangeMillis(100) .withEnableTaskAutoScaler(true) - .withScaleActionPeriodMillis(3000) + .withScaleActionPeriodMillis(60000) .withScaleActionStartDelayMillis(1000) .withScaleOutThreshold(0) .withScaleInThreshold(10000) @@ -473,7 +473,9 @@ private KafkaSupervisorSpec createKafkaSupervisor( return MoreResources.Supervisor.KAFKA_JSON .get() .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) - .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(maxRowsPerSegment)) + .withTuningConfig(tuningConfig -> tuningConfig + .withMaxRowsPerSegment(maxRowsPerSegment) + .withReleaseLocksOnHandoff(true)) .withIoConfig( ioConfig -> ioConfig .withConsumerProperties(kafkaServer.consumerProperties()) 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 eba5edfb71a1..b4a5a14b8440 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 @@ -406,7 +406,8 @@ protected Map getLatestSequencesFromStream() protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( Set partitions, SeekableStreamSupervisor.TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ) { throw new UnsupportedOperationException("not implemented"); 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 0ea0053988cb..bb3ab40e0d3e 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 @@ -256,7 +256,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( Set partitions, TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ) { log.info("Creating updated task IO config for task group [%s]", existingTaskGroup.getId()); @@ -264,14 +265,10 @@ protected SeekableStreamIndexTaskIOConfig createUpdat Set exclusiveStartSequenceNumberPartitions = new HashSet<>(); for (KafkaTopicPartition partition : partitions) { - Long offset; - if (!latestCommittedOffsets.containsKey(partition)) { - log.warn("No committed offset found for partition [%s], using NOT_SET", partition); - offset = NOT_SET; - } else { - offset = latestCommittedOffsets.get(partition); - } - + Long offset = Math.max( + latestTaskOffsetsOnPause.getOrDefault(partition, 0L), + latestCommittedOffsets.getOrDefault(partition, 0L) + ); startingSequences.put(partition, offset); } @@ -285,7 +282,7 @@ protected SeekableStreamIndexTaskIOConfig createUpdat // For end sequences, use NOT_SET to indicate open-ended reading Map endingSequences = new HashMap<>(); for (KafkaTopicPartition partition : partitions) { - endingSequences.put(partition, NOT_SET); + endingSequences.put(partition, END_OF_PARTITION); } SeekableStreamEndSequenceNumbers endSequenceNumbers = 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 cf22d06dee41..f491db05b412 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 @@ -153,7 +153,8 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( Set partitions, TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ) { throw new UnsupportedOperationException("Not implemented"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 92e9ff2b0634..3bdcf91004a2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -91,6 +91,8 @@ ListenableFuture>> get */ ListenableFuture> pauseAsync(String id); + ListenableFuture> pauseAndCheckpointAsync(String id); + /** * Set end offsets for a task. Retries on failure. * diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 447e9ca27245..ee8bc28ff28f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -325,6 +325,51 @@ public ListenableFuture> pauseAsync(Str ); } + @Override + public ListenableFuture> pauseAndCheckpointAsync(String id) + { + final ListenableFuture> pauseFuture = + makeRequest(id, new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint")) + .handler(new BytesFullResponseHandler()) + .onSuccess(r -> { + if (r.getStatus().equals(HttpResponseStatus.OK)) { + log.info("Task [%s] paused successfully & Checkpoint requested succesffully", id); + return deserializeOffsetsMap(r.getContent()); + } else if (r.getStatus().equals(HttpResponseStatus.ACCEPTED)) { + // Return null, which triggers a loop later to wait for the task to enter PAUSED state. + return null; + } else { + throw new ISE( + "Pause & Checkpoint request for task [%s] failed with response [%s]", + id, + r.getStatus() + ); + } + }) + .onNotAvailable(e -> Either.value(Collections.emptyMap())) + .go(); + + return FutureUtils.transformAsync( + pauseFuture, + result -> { + if (result != null) { + return Futures.immediateFuture(result); + } else { + return getOffsetsWhenPaused( + id, + new RetryPolicyFactory( + new RetryPolicyConfig() + .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS)) + .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS)) + .setMaxRetryCount(httpRetries) + ).makeRetryPolicy() + ); + } + } + ); + } + + @Override public ListenableFuture> getMovingAveragesAsync(String id) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 555af51bcbed..49ac77e16d7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -119,7 +119,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -254,7 +253,7 @@ public enum Status private volatile DateTime minMessageTime; private volatile DateTime maxMessageTime; private final ScheduledExecutorService rejectionPeriodUpdaterExec; - private volatile boolean isConfigChangeOngoing = false; + private volatile boolean isPausedForCheckpointCompletion = false; public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, @@ -1738,27 +1737,20 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt throws InterruptedException { authorizationCheck(req); + if (this.isPausedForCheckpointCompletion) { + return Response.status(409).entity("Another config change is ongoing").build(); + } try { log.info("Attempting to update config to [%s]", request.getIoConfig()); pause(); - isConfigChangeOngoing = true; + this.isPausedForCheckpointCompletion = true; checkpointSequences(); - SeekableStreamIndexTaskIOConfig newIoConfig; - if (request.getIoConfig() instanceof LinkedHashMap) { - newIoConfig =(SeekableStreamIndexTaskIOConfig) - toolbox.getJsonMapper().convertValue(request.getIoConfig(), SeekableStreamIndexTaskIOConfig.class); - } else { - newIoConfig = (SeekableStreamIndexTaskIOConfig) request.getIoConfig(); - } - this.ioConfig = newIoConfig; - this.stream = ioConfig.getStartSequenceNumbers().getStream(); - this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); - minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); - maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); - + SeekableStreamIndexTaskIOConfig newIoConfig = (SeekableStreamIndexTaskIOConfig) + toolbox.getJsonMapper().convertValue(request.getIoConfig(), SeekableStreamIndexTaskIOConfig.class); + setIOConfig(newIoConfig); createNewSequenceFromIoConfig(newIoConfig); - isConfigChangeOngoing = false; + this.isPausedForCheckpointCompletion = false; log.info("Config updated to [%s]", this.ioConfig); toolbox.getEmitter().emit(ServiceMetricEvent.builder() .setDimension(DruidMetrics.TASK_ID, task.getId()) @@ -1766,14 +1758,27 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) .build("task/config/update/success", String.valueOf(1))); resume(); + this.isPausedForCheckpointCompletion = false; return Response.ok().build(); } catch (Exception e) { log.makeAlert(e, "Failed to update task config"); + this.isPausedForCheckpointCompletion = false; return Response.serverError().entity(e.getMessage()).build(); } } + private void setIOConfig( + SeekableStreamIndexTaskIOConfig ioConfig + ) throws IOException + { + this.ioConfig = ioConfig; + this.stream = ioConfig.getStartSequenceNumbers().getStream(); + this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); + this.minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); + this.maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); + } + /** * Creates new sequences for the ingestion process. It currently accepts the ioConfig given by the request as the correct offsets * and ignores the offsets it may have stored in currOffsets and endOffsets. @@ -1862,8 +1867,8 @@ public Response setEndOffsets( .build(); } else { try { - // Don't acquire a lock if a config change is ongoing, as the runner is already paused. - if (!isConfigChangeOngoing) { + // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock + if (!this.isPausedForCheckpointCompletion) { pauseLock.lockInterruptibly(); } // Perform all sequence related checks before checking for isPaused() @@ -1889,7 +1894,7 @@ public Response setEndOffsets( || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resetNextCheckpointTime(); - if (!isConfigChangeOngoing) { + if (!this.isPausedForCheckpointCompletion) { resume(); } return Response.ok(sequenceNumbers).build(); @@ -1925,7 +1930,9 @@ public Response setEndOffsets( resetNextCheckpointTime(); latestSequence.setEndOffsets(sequenceNumbers); - if (finish) { + // if this is the final checkpoint, or if the task is paused for checkpoint completion and updateConfig is supposed to + // finish the current sequence, we just update the end offsets of the latest sequence. + if (finish || this.isPausedForCheckpointCompletion) { log.info( "Sequence[%s] end offsets updated from [%s] to [%s].", latestSequence.getSequenceName(), @@ -1966,11 +1973,13 @@ public Response setEndOffsets( .build(); } finally { - pauseLock.unlock(); + if (!this.isPausedForCheckpointCompletion) { + pauseLock.unlock(); + } } } - if (!isConfigChangeOngoing) { + if (!isPausedForCheckpointCompletion) { resume(); } @@ -2026,6 +2035,21 @@ public Response pauseHTTP( return pause(); } + @POST + @Path("/pauseAndCheckpoint") + @Produces(MediaType.APPLICATION_JSON) + public Response pauseAndCheckpointHTTP( + @Context final HttpServletRequest req + ) throws InterruptedException, JsonProcessingException + { + authorizationCheck(req); + log.info("Received pause and checkpoint request"); + pause(); + this.isPausedForCheckpointCompletion = true; + checkpointSequences(); + return Response.ok().entity(toolbox.getJsonMapper().writeValueAsString(getCurrentOffsets())).build(); + } + @VisibleForTesting public Response pause() throws InterruptedException { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java index 42029da70e19..abb253cd7d5a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -32,18 +32,18 @@ */ public class TaskConfigUpdateRequest { - private final Object ioConfig; + private final SeekableStreamIndexTaskIOConfig ioConfig; @JsonCreator public TaskConfigUpdateRequest( - @JsonProperty("ioConfig") @Nullable Object ioConfig + @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig ) { this.ioConfig = ioConfig; } @JsonProperty - public Object getIoConfig() + public SeekableStreamIndexTaskIOConfig getIoConfig() { return ioConfig; } 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 8fee67b06bf9..6c1dbc74a83a 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 @@ -124,6 +124,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; @@ -584,7 +585,7 @@ private boolean changeTaskCount(int desiredActiveTaskCount) final Stopwatch scaleActionStopwatch = Stopwatch.createStarted(); if (spec.usePerpetuallyRunningTasks()) { - return changeTaskCountForPerpetualTasks(desiredActiveTaskCount, scaleActionStopwatch); + return changeTaskCountForPerpetualTasks(desiredActiveTaskCount); } else { gracefulShutdownInternal(); changeTaskCountInIOConfig(desiredActiveTaskCount); @@ -629,17 +630,27 @@ private Map getLatestOffsetsFromMetadataSto * Handles task count changes for perpetual tasks using updateConfig instead of graceful shutdown. * This approach pauses tasks, recalculates partition assignments, and sends config updates. */ - private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount, Stopwatch scaleActionStopwatch) + private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount) throws InterruptedException, ExecutionException { + if (this.isDynamicAllocationOngoing) { + log.info("A dynamic allocation is already ongoing, skipping this request."); + return false; + } + this.isDynamicAllocationOngoing = true; log.info("Handling task count change for perpetual tasks from [%d] to [%d]", activelyReadingTaskGroups.size(), desiredActiveTaskCount); - pauseAllTasks(); + Map offsetsFromTasks = pauseAndCheckpointAllTasks(); + if (taskCheckpointWait.await(60, TimeUnit.SECONDS)) { + log.info("All tasks have been checkpointed successfully"); + } else { + log.warn("Timeout waiting for tasks to pause and checkpoint"); + } changeTaskCountInIOConfig(desiredActiveTaskCount); Map> newPartitionGroups = calculateNewPartitionGroups(); - boolean success = sendConfigUpdatesToTasks(newPartitionGroups); + boolean success = sendConfigUpdatesToTasks(newPartitionGroups, offsetsFromTasks); if (success) { updatePartitionGroupsForPerpetualTasks(newPartitionGroups); @@ -647,11 +658,11 @@ private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount, Sto } else { log.error("Failed to update task configurations for perpetual tasks"); } - + this.isDynamicAllocationOngoing = false; return success; } - private void pauseAllTasks() throws InterruptedException, ExecutionException + private Map pauseAndCheckpointAllTasks() throws InterruptedException, ExecutionException { log.info("Pausing all tasks for perpetual task scaling"); List>> pauseFutures = new ArrayList<>(); @@ -659,12 +670,32 @@ private void pauseAllTasks() throws InterruptedException, ExecutionException for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (String taskId : taskGroup.taskIds()) { log.debug("Pausing task [%s]", taskId); - pauseFutures.add(taskClient.pauseAsync(taskId)); + pauseFutures.add(taskClient.pauseAndCheckpointAsync(taskId)); } } - - coalesceAndAwait(pauseFutures); - log.info("Successfully paused [%d] tasks", pauseFutures.size()); + + Map consolidatedLatestOffsets = new HashMap<>(); + + this.taskCheckpointWait = new CountDownLatch(pauseFutures.size()); + log.info("Staring a task checkpoint wait with count [%d]", pauseFutures.size()); + List>> latestOffsets = coalesceAndAwait(pauseFutures); + for (Either> result : latestOffsets) { + if (result.isError()) { + log.warn("Failed to pause and checkpoint task[%s]", result.error().getMessage()); + } else { + Map offsets = result.valueOrThrow(); + offsets.forEach((partition, offset) -> + consolidatedLatestOffsets.merge( + partition, offset, (key, existingOffset) -> { + OrderedSequenceNumber existing = makeSequenceNumber(existingOffset); + OrderedSequenceNumber incoming = makeSequenceNumber(offset); + return existing.compareTo(incoming) >= 0 ? existingOffset : offset; + } + )); + } + } + log.info("Successfully paused & requested checkpointed from [%d] tasks", pauseFutures.size()); + return consolidatedLatestOffsets; } /** @@ -700,7 +731,7 @@ private Map> calculateNewPartitionGroups() * Sends configuration updates to tasks with new partition assignments. * Also handles cleanup of obsolete task groups when scaling down. */ - private boolean sendConfigUpdatesToTasks(Map> newPartitionGroups) + private boolean sendConfigUpdatesToTasks(Map> newPartitionGroups, Map latestTaskOffsetsOnPause) throws InterruptedException, ExecutionException { log.info("Sending configuration updates to tasks"); @@ -716,7 +747,8 @@ private boolean sendConfigUpdatesToTasks(Map> newP SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig( partitions, existingTaskGroup, - latestCommittedOffsets + latestCommittedOffsets, + latestTaskOffsetsOnPause ); TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); @@ -817,7 +849,8 @@ private void updatePartitionGroupsForPerpetualTasks(Map createUpdatedTaskIoConfig( Set partitions, TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ); private void changeTaskCountInIOConfig(int desiredActiveTaskCount) @@ -1084,6 +1117,8 @@ public String getType() private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; protected final String dataSource; + private volatile CountDownLatch taskCheckpointWait; + private volatile boolean isDynamicAllocationOngoing; private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; @@ -3765,6 +3800,8 @@ public ListenableFuture> apply(List> apply(List stopTasksInGroup( @Nullable TaskGroup taskGroup, String stopReasonFormat, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index a5f66e794ff8..fc30c5bee6a1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -369,7 +369,8 @@ public LagStats computeLagStats() protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( Set partitions, SeekableStreamSupervisor.TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ) { // dummy implementation 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 f7968ae07e1d..59b8c84e5a24 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 @@ -2952,7 +2952,8 @@ private BaseTestSeekableStreamSupervisor() protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( Set partitions, SeekableStreamSupervisor.TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ) { // dummy implementation @@ -3178,7 +3179,8 @@ public LagStats computeLagStats() protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( Set partitions, SeekableStreamSupervisor.TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ) { return null; @@ -3225,7 +3227,8 @@ private class TestEmittingTestSeekableStreamSupervisor extends BaseTestSeekableS protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( Set partitions, SeekableStreamSupervisor.TaskGroup existingTaskGroup, - Map latestCommittedOffsets + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause ) { return null; From 4b51c147dd35d559bc92213e28b4f375e2e95611 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 11 Sep 2025 11:47:25 +0530 Subject: [PATCH 16/46] Fix latches and clean up maven issues --- .../indexing/KafkaClusterMetricsTest.java | 18 +++-- .../kafka/supervisor/KafkaSupervisor.java | 4 +- .../supervisor/SupervisorManager.java | 1 + ...eekableStreamIndexTaskClientAsyncImpl.java | 1 - .../SeekableStreamIndexTaskRunner.java | 29 ++++---- .../TaskConfigUpdateRequest.java | 2 - .../supervisor/SeekableStreamSupervisor.java | 73 +++++++++++-------- 7 files changed, 71 insertions(+), 57 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 6cfc17656d4e..a209a8ca5d84 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -27,7 +27,6 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; import org.apache.druid.indexing.kafka.simulate.KafkaResource; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; -import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpecBuilder; import org.apache.druid.indexing.kafka.supervisor.LagBasedAutoScalerConfigBuilder; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; @@ -102,8 +101,10 @@ public void stop() }; indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.server.http.numThreads", "30") .addProperty("druid.worker.capacity", "10"); overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.server.http.numThreads", "50") .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") .addProperty("druid.manager.segments.pollDuration", "PT0.1s") .addProperty("druid.manager.segments.killUnused.enabled", "true") @@ -181,11 +182,11 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() } @Test - @Timeout(60) - public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() + @Timeout(600) + public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() { final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 50; + final int expectedSegmentsHandedOff = 20; final int taskCount = 1; @@ -196,7 +197,7 @@ public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues .withLagCollectionRangeMillis(100) .withEnableTaskAutoScaler(true) .withScaleActionPeriodMillis(60000) - .withScaleActionStartDelayMillis(1000) + .withScaleActionStartDelayMillis(100) .withScaleOutThreshold(0) .withScaleInThreshold(10000) .withTriggerScaleOutFractionThreshold(0.001) @@ -221,6 +222,13 @@ public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues cluster.callApi().postSupervisor(kafkaSupervisorSpec) ); + overlord.latchableEmitter().waitForEvent(event -> event.hasMetricName("task/autoScaler/requiredCount")); + + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("task/autoScaler/scaleActionTime") + .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)) + ); + indexer.latchableEmitter().waitForEventAggregate( event -> event.hasMetricName("ingest/handoff/count") .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)), 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 bb3ab40e0d3e..6a121b99c2bd 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 @@ -76,8 +76,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -import static java.lang.Math.max; - /** * Supervisor responsible for managing the KafkaIndexTasks for a single dataSource. At a high level, the class accepts a * {@link KafkaSupervisorSpec} which includes the Kafka topic and configuration as well as an ingestion spec which will @@ -170,7 +168,7 @@ private int getRangeBasedTaskGroupId(KafkaTopicPartition partitionId, Integer ta if (partitionId.isMultiTopicPartition()) { return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; } else { - return max(taskCount - 1, partitionId.partition() / minPartitionsPerTaskGroup); + return max(taskCount - 1, (partitionId.partition() / minPartitionsPerTaskGroup) - 1); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 168b956afd2d..162902b036c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -372,6 +372,7 @@ public boolean checkPointDataSourceMetadata( Preconditions.checkNotNull(supervisor, "supervisor could not be found"); final StreamSupervisor streamSupervisor = requireStreamSupervisor(supervisorId, "checkPoint"); + log.info("Checkpointing datasource metadata for supervisor [%s]", supervisorId); streamSupervisor.checkpoint(taskGroupId, previousDataSourceMetadata); return true; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index ee8bc28ff28f..2d5d2b58cbd2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -215,7 +215,6 @@ public ListenableFuture updateConfigAsync(String taskId, TaskConfigUpda { final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/updateConfig") .jsonContent(jsonMapper, updateRequest); - return makeRequest(taskId, requestBuilder) .handler(IgnoreHttpResponseHandler.INSTANCE) .onSuccess(r -> true) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 49ac77e16d7b..6efdbb97ff1c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1737,33 +1737,32 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt throws InterruptedException { authorizationCheck(req); - if (this.isPausedForCheckpointCompletion) { - return Response.status(409).entity("Another config change is ongoing").build(); + if (!isPausedForCheckpointCompletion) { + return Response.status(409).entity("Task must be paused for checkpoint completion before updating config").build(); } try { log.info("Attempting to update config to [%s]", request.getIoConfig()); - pause(); - this.isPausedForCheckpointCompletion = true; - checkpointSequences(); + System.out.printf("Attempting to update config to [%s]%n", request.getIoConfig()); SeekableStreamIndexTaskIOConfig newIoConfig = (SeekableStreamIndexTaskIOConfig) toolbox.getJsonMapper().convertValue(request.getIoConfig(), SeekableStreamIndexTaskIOConfig.class); setIOConfig(newIoConfig); createNewSequenceFromIoConfig(newIoConfig); - this.isPausedForCheckpointCompletion = false; + log.info("Config updated to [%s]", this.ioConfig); + System.out.printf("Config updated to [%s]%n", this.ioConfig); toolbox.getEmitter().emit(ServiceMetricEvent.builder() .setDimension(DruidMetrics.TASK_ID, task.getId()) .setDimension(DruidMetrics.TASK_TYPE, task.getType()) .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) .build("task/config/update/success", String.valueOf(1))); resume(); - this.isPausedForCheckpointCompletion = false; + isPausedForCheckpointCompletion = false; return Response.ok().build(); } catch (Exception e) { log.makeAlert(e, "Failed to update task config"); - this.isPausedForCheckpointCompletion = false; + isPausedForCheckpointCompletion = false; return Response.serverError().entity(e.getMessage()).build(); } } @@ -1868,7 +1867,7 @@ public Response setEndOffsets( } else { try { // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock - if (!this.isPausedForCheckpointCompletion) { + if (!isPausedForCheckpointCompletion) { pauseLock.lockInterruptibly(); } // Perform all sequence related checks before checking for isPaused() @@ -1894,7 +1893,7 @@ public Response setEndOffsets( || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resetNextCheckpointTime(); - if (!this.isPausedForCheckpointCompletion) { + if (!isPausedForCheckpointCompletion) { resume(); } return Response.ok(sequenceNumbers).build(); @@ -1932,7 +1931,7 @@ public Response setEndOffsets( // if this is the final checkpoint, or if the task is paused for checkpoint completion and updateConfig is supposed to // finish the current sequence, we just update the end offsets of the latest sequence. - if (finish || this.isPausedForCheckpointCompletion) { + if (finish || isPausedForCheckpointCompletion) { log.info( "Sequence[%s] end offsets updated from [%s] to [%s].", latestSequence.getSequenceName(), @@ -1973,7 +1972,7 @@ public Response setEndOffsets( .build(); } finally { - if (!this.isPausedForCheckpointCompletion) { + if (!isPausedForCheckpointCompletion) { pauseLock.unlock(); } } @@ -2043,9 +2042,11 @@ public Response pauseAndCheckpointHTTP( ) throws InterruptedException, JsonProcessingException { authorizationCheck(req); - log.info("Received pause and checkpoint request"); + if (isPausedForCheckpointCompletion) { + return Response.ok().entity("Task is already paused for checkpoint completion").build(); + } pause(); - this.isPausedForCheckpointCompletion = true; + isPausedForCheckpointCompletion = true; checkpointSequences(); return Response.ok().entity(toolbox.getJsonMapper().writeValueAsString(getCurrentOffsets())).build(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java index abb253cd7d5a..3515da252c99 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -21,10 +21,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; import javax.annotation.Nullable; -import java.util.LinkedHashMap; import java.util.Objects; /** 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 6c1dbc74a83a..1cb541f535ee 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 @@ -175,6 +175,7 @@ public abstract class SeekableStreamSupervisor getLatestOffsetsFromMetadataStore() { try { @@ -633,16 +638,18 @@ private Map getLatestOffsetsFromMetadataSto private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount) throws InterruptedException, ExecutionException { - if (this.isDynamicAllocationOngoing) { + if (isDynamicAllocationOngoing) { log.info("A dynamic allocation is already ongoing, skipping this request."); return false; } - this.isDynamicAllocationOngoing = true; + final Stopwatch scaleActionStopwatch = Stopwatch.createStarted(); + isDynamicAllocationOngoing = true; log.info("Handling task count change for perpetual tasks from [%d] to [%d]", activelyReadingTaskGroups.size(), desiredActiveTaskCount); Map offsetsFromTasks = pauseAndCheckpointAllTasks(); - if (taskCheckpointWait.await(60, TimeUnit.SECONDS)) { + final CountDownLatch currentLatch = taskCheckpointWait; + if (currentLatch.await(CHECKPOINT_COMPLETION_LATCH_TIMEOUT, TimeUnit.SECONDS)) { log.info("All tasks have been checkpointed successfully"); } else { log.warn("Timeout waiting for tasks to pause and checkpoint"); @@ -659,6 +666,7 @@ private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount) log.error("Failed to update task configurations for perpetual tasks"); } this.isDynamicAllocationOngoing = false; + emitAutoScalerRunMetric(scaleActionStopwatch); return success; } @@ -669,14 +677,14 @@ private Map pauseAndCheckpointAllTasks() th for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { for (String taskId : taskGroup.taskIds()) { - log.debug("Pausing task [%s]", taskId); + log.debug("Pausing & Checkpointing tasks [%s]", taskId); pauseFutures.add(taskClient.pauseAndCheckpointAsync(taskId)); } } Map consolidatedLatestOffsets = new HashMap<>(); - this.taskCheckpointWait = new CountDownLatch(pauseFutures.size()); + taskCheckpointWait = new CountDownLatch(pauseFutures.size()); log.info("Staring a task checkpoint wait with count [%d]", pauseFutures.size()); List>> latestOffsets = coalesceAndAwait(pauseFutures); for (Either> result : latestOffsets) { @@ -686,7 +694,9 @@ private Map pauseAndCheckpointAllTasks() th Map offsets = result.valueOrThrow(); offsets.forEach((partition, offset) -> consolidatedLatestOffsets.merge( - partition, offset, (key, existingOffset) -> { + partition, + offset, + (key, existingOffset) -> { OrderedSequenceNumber existing = makeSequenceNumber(existingOffset); OrderedSequenceNumber incoming = makeSequenceNumber(offset); return existing.compareTo(incoming) >= 0 ? existingOffset : offset; @@ -715,12 +725,8 @@ private Map> calculateNewPartitionGroups() for (PartitionIdType partition : allPartitions) { int taskGroupId = getTaskGroupIdForPartition(partition); - - if (taskGroupId >= 0 && taskGroupId < ioConfig.getTaskCount()) { - newPartitionGroups.computeIfAbsent(taskGroupId, k -> new HashSet<>()).add(partition); - } else { - log.warn("Invalid task group ID [%d] for partition [%s], skipping", taskGroupId, partition); - } + newPartitionGroups.computeIfAbsent(taskGroupId, k -> new HashSet<>()).add(partition); + log.info("received taskGroupid [%d] and newPartitionGroup so far: %s", taskGroupId, newPartitionGroups); } log.info("Created [%d] new partition groups: %s", newPartitionGroups.size(), newPartitionGroups); @@ -734,7 +740,8 @@ private Map> calculateNewPartitionGroups() private boolean sendConfigUpdatesToTasks(Map> newPartitionGroups, Map latestTaskOffsetsOnPause) throws InterruptedException, ExecutionException { - log.info("Sending configuration updates to tasks"); + log.info("Sending configuration updates to the following partition groups %s", newPartitionGroups); + log.info("Actively reading task groups right now %s", activelyReadingTaskGroups); List> updateFutures = new ArrayList<>(); Map latestCommittedOffsets = getLatestOffsetsFromMetadataStore(); for (Map.Entry> entry : newPartitionGroups.entrySet()) { @@ -1117,7 +1124,7 @@ public String getType() private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; protected final String dataSource; - private volatile CountDownLatch taskCheckpointWait; + private volatile CountDownLatch taskCheckpointWait = new CountDownLatch(0); private volatile boolean isDynamicAllocationOngoing; private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); @@ -3793,7 +3800,7 @@ public ListenableFuture> apply(List> apply(List> apply(List stopTasksInGroup( @Nullable TaskGroup taskGroup, From dcd3549c0121424017984a2d900364b4f8cae513 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 11 Sep 2025 13:00:07 +0530 Subject: [PATCH 17/46] Use atomic booleans for coordination b/w multiple triggers of auto scaling event --- .../kafka/supervisor/KafkaSupervisor.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 24 +++++----- .../supervisor/SeekableStreamSupervisor.java | 47 ++++++------------- 3 files changed, 27 insertions(+), 46 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 6a121b99c2bd..c063087eeb3c 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 @@ -168,7 +168,7 @@ private int getRangeBasedTaskGroupId(KafkaTopicPartition partitionId, Integer ta if (partitionId.isMultiTopicPartition()) { return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; } else { - return max(taskCount - 1, (partitionId.partition() / minPartitionsPerTaskGroup) - 1); + return Math.max(taskCount - 1, (partitionId.partition() / minPartitionsPerTaskGroup) - 1); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 6efdbb97ff1c..eff5e5c26b7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -253,7 +253,7 @@ public enum Status private volatile DateTime minMessageTime; private volatile DateTime maxMessageTime; private final ScheduledExecutorService rejectionPeriodUpdaterExec; - private volatile boolean isPausedForCheckpointCompletion = false; + private volatile boolean waitForConfigUpdate = false; public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, @@ -1737,12 +1737,11 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt throws InterruptedException { authorizationCheck(req); - if (!isPausedForCheckpointCompletion) { + if (!waitForConfigUpdate) { return Response.status(409).entity("Task must be paused for checkpoint completion before updating config").build(); } try { log.info("Attempting to update config to [%s]", request.getIoConfig()); - System.out.printf("Attempting to update config to [%s]%n", request.getIoConfig()); SeekableStreamIndexTaskIOConfig newIoConfig = (SeekableStreamIndexTaskIOConfig) toolbox.getJsonMapper().convertValue(request.getIoConfig(), SeekableStreamIndexTaskIOConfig.class); @@ -1750,19 +1749,18 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt createNewSequenceFromIoConfig(newIoConfig); log.info("Config updated to [%s]", this.ioConfig); - System.out.printf("Config updated to [%s]%n", this.ioConfig); toolbox.getEmitter().emit(ServiceMetricEvent.builder() .setDimension(DruidMetrics.TASK_ID, task.getId()) .setDimension(DruidMetrics.TASK_TYPE, task.getType()) .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) .build("task/config/update/success", String.valueOf(1))); resume(); - isPausedForCheckpointCompletion = false; + waitForConfigUpdate = false; return Response.ok().build(); } catch (Exception e) { log.makeAlert(e, "Failed to update task config"); - isPausedForCheckpointCompletion = false; + waitForConfigUpdate = false; return Response.serverError().entity(e.getMessage()).build(); } } @@ -1867,7 +1865,7 @@ public Response setEndOffsets( } else { try { // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock - if (!isPausedForCheckpointCompletion) { + if (!waitForConfigUpdate) { pauseLock.lockInterruptibly(); } // Perform all sequence related checks before checking for isPaused() @@ -1893,7 +1891,7 @@ public Response setEndOffsets( || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resetNextCheckpointTime(); - if (!isPausedForCheckpointCompletion) { + if (!waitForConfigUpdate) { resume(); } return Response.ok(sequenceNumbers).build(); @@ -1931,7 +1929,7 @@ public Response setEndOffsets( // if this is the final checkpoint, or if the task is paused for checkpoint completion and updateConfig is supposed to // finish the current sequence, we just update the end offsets of the latest sequence. - if (finish || isPausedForCheckpointCompletion) { + if (finish || waitForConfigUpdate) { log.info( "Sequence[%s] end offsets updated from [%s] to [%s].", latestSequence.getSequenceName(), @@ -1972,13 +1970,13 @@ public Response setEndOffsets( .build(); } finally { - if (!isPausedForCheckpointCompletion) { + if (!waitForConfigUpdate) { pauseLock.unlock(); } } } - if (!isPausedForCheckpointCompletion) { + if (!waitForConfigUpdate) { resume(); } @@ -2042,11 +2040,11 @@ public Response pauseAndCheckpointHTTP( ) throws InterruptedException, JsonProcessingException { authorizationCheck(req); - if (isPausedForCheckpointCompletion) { + if (waitForConfigUpdate) { return Response.ok().entity("Task is already paused for checkpoint completion").build(); } pause(); - isPausedForCheckpointCompletion = true; + waitForConfigUpdate = true; checkpointSequences(); return Response.ok().entity(toolbox.getJsonMapper().writeValueAsString(getCurrentOffsets())).build(); } 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 1cb541f535ee..49651c69c3a8 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 @@ -130,6 +130,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; @@ -614,23 +615,6 @@ private void emitAutoScalerRunMetric(Stopwatch scaleActionStopwatch) )); } - private Map getLatestOffsetsFromMetadataStore() - { - try { - DataSourceMetadata metadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(dataSource); - if (metadata instanceof SeekableStreamDataSourceMetadata) { - @SuppressWarnings("unchecked") - SeekableStreamDataSourceMetadata streamMetadata = - (SeekableStreamDataSourceMetadata) metadata; - return streamMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap(); - } - } - catch (Exception e) { - log.warn(e, "Failed to retrieve latest offsets from metadata store, using current partition state"); - } - return Collections.emptyMap(); - } - /** * Handles task count changes for perpetual tasks using updateConfig instead of graceful shutdown. * This approach pauses tasks, recalculates partition assignments, and sends config updates. @@ -638,17 +622,16 @@ private Map getLatestOffsetsFromMetadataSto private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount) throws InterruptedException, ExecutionException { - if (isDynamicAllocationOngoing) { + if (!isDynamicAllocationOngoing.compareAndSet(false, true)) { log.info("A dynamic allocation is already ongoing, skipping this request."); return false; } final Stopwatch scaleActionStopwatch = Stopwatch.createStarted(); - isDynamicAllocationOngoing = true; - log.info("Handling task count change for perpetual tasks from [%d] to [%d]", + log.info("Handling task count change for perpetual tasks from [%d] to [%d]", activelyReadingTaskGroups.size(), desiredActiveTaskCount); Map offsetsFromTasks = pauseAndCheckpointAllTasks(); - final CountDownLatch currentLatch = taskCheckpointWait; + final CountDownLatch currentLatch = checkpointsForConfigUpdateLatch; if (currentLatch.await(CHECKPOINT_COMPLETION_LATCH_TIMEOUT, TimeUnit.SECONDS)) { log.info("All tasks have been checkpointed successfully"); } else { @@ -665,7 +648,7 @@ private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount) } else { log.error("Failed to update task configurations for perpetual tasks"); } - this.isDynamicAllocationOngoing = false; + isDynamicAllocationOngoing.set(false); emitAutoScalerRunMetric(scaleActionStopwatch); return success; } @@ -684,7 +667,7 @@ private Map pauseAndCheckpointAllTasks() th Map consolidatedLatestOffsets = new HashMap<>(); - taskCheckpointWait = new CountDownLatch(pauseFutures.size()); + checkpointsForConfigUpdateLatch = new CountDownLatch(pauseFutures.size()); log.info("Staring a task checkpoint wait with count [%d]", pauseFutures.size()); List>> latestOffsets = coalesceAndAwait(pauseFutures); for (Either> result : latestOffsets) { @@ -704,12 +687,12 @@ private Map pauseAndCheckpointAllTasks() th )); } } - log.info("Successfully paused & requested checkpointed from [%d] tasks", pauseFutures.size()); + log.info("Successfully paused & checkpoints requested from [%d] tasks", pauseFutures.size()); return consolidatedLatestOffsets; } /** - * Calculates new partition groups using the existing getTaskGroupIdForPartition() logic. + * Calculates new partition groups. */ private Map> calculateNewPartitionGroups() { @@ -741,9 +724,8 @@ private boolean sendConfigUpdatesToTasks(Map> newP throws InterruptedException, ExecutionException { log.info("Sending configuration updates to the following partition groups %s", newPartitionGroups); - log.info("Actively reading task groups right now %s", activelyReadingTaskGroups); List> updateFutures = new ArrayList<>(); - Map latestCommittedOffsets = getLatestOffsetsFromMetadataStore(); + Map latestCommittedOffsets = getOffsetsFromMetadataStorage(); for (Map.Entry> entry : newPartitionGroups.entrySet()) { int taskGroupId = entry.getKey(); Set partitions = entry.getValue(); @@ -1124,8 +1106,8 @@ public String getType() private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; protected final String dataSource; - private volatile CountDownLatch taskCheckpointWait = new CountDownLatch(0); - private volatile boolean isDynamicAllocationOngoing; + private volatile CountDownLatch checkpointsForConfigUpdateLatch; + private final AtomicBoolean isDynamicAllocationOngoing = new AtomicBoolean(false); private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; @@ -3831,11 +3813,12 @@ public ListenableFuture> apply(List stopTasksInGroup( @Nullable TaskGroup taskGroup, String stopReasonFormat, From 673f1147669fcffc558dafa44a4596d43c18d8be Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 11 Sep 2025 23:06:24 +0530 Subject: [PATCH 18/46] Fix range based partition assignment logic --- .../kafka/supervisor/KafkaSupervisor.java | 27 +++++--- .../kafka/supervisor/KafkaSupervisorTest.java | 65 ++++++++++++++++++- 2 files changed, 81 insertions(+), 11 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index c063087eeb3c..0f1e5cc88067 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 @@ -144,7 +144,7 @@ protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) if (spec.usePerpetuallyRunningTasks()) { int taskGroupId = getRangeBasedTaskGroupId(partitionId, taskCount); - log.debug("Range-based assignment for partition [%s]: taskGroupId [%d]", partitionId, taskGroupId); + log.debug("Range-based assignment for partition [%s]: taskGroupId [%d] when taskCount is [%d]", partitionId, taskGroupId, taskCount); return taskGroupId; } else { if (partitionId.isMultiTopicPartition()) { @@ -155,21 +155,28 @@ protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) } } - /** - * Assigns partitions to task groups using range-based sequential assignment. - * This ensures that adjacent partitions are assigned to the same task group - */ - private int getRangeBasedTaskGroupId(KafkaTopicPartition partitionId, Integer taskCount) + @VisibleForTesting + public int getRangeBasedTaskGroupIdForPartition(KafkaTopicPartition partitionId, Integer taskCount, int totalPartitions) { - int totalPartitions = partitionIds.size(); int minPartitionsPerTaskGroup = totalPartitions / taskCount; - if (partitionId.isMultiTopicPartition()) { return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; - } else { - return Math.max(taskCount - 1, (partitionId.partition() / minPartitionsPerTaskGroup) - 1); } + int taskGroup = (partitionId.partition() / minPartitionsPerTaskGroup); + if (taskGroup >= taskCount) { + taskGroup--; + } + return taskGroup; + } + + /** + * Assigns partitions to task groups using range-based sequential assignment. + * This ensures that adjacent partitions are assigned to the same task group + */ + private int getRangeBasedTaskGroupId(KafkaTopicPartition partitionId, Integer taskCount) + { + return getRangeBasedTaskGroupIdForPartition(partitionId, taskCount, partitionIds.size()); } @Override 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 5ad9867cb40f..2ed14296f273 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 @@ -5042,6 +5042,38 @@ public void test_doesTaskMatchSupervisor() EasyMock.replay(differentTaskType); } + @Test + public void test_getTaskGroupIdForPartition() throws Exception + { + int taskCount = 3; + supervisor = getTestableSupervisor( + "test-supervisor", + 1, + taskCount, + true, + false, + "PT1H", + null, + null, + false, + kafkaHost, + null, + true + ); + + int totalPartitions = 10; + Assert.assertEquals(0, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 0), taskCount, totalPartitions)); + Assert.assertEquals(0, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 1), taskCount, totalPartitions)); + Assert.assertEquals(0, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 2), taskCount, totalPartitions)); + Assert.assertEquals(1, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 3), taskCount, totalPartitions)); + Assert.assertEquals(1, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 4), taskCount, totalPartitions)); + Assert.assertEquals(1, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 5), taskCount, totalPartitions)); + Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 6), taskCount, totalPartitions)); + Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 7), taskCount, totalPartitions)); + Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 8), taskCount, totalPartitions)); + Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 9), taskCount, totalPartitions)); + } + private void addSomeEvents(int numEventsPerPartition) throws Exception { // create topic manually @@ -5222,6 +5254,37 @@ private TestableKafkaSupervisor getTestableSupervisor( String kafkaHost, IdleConfig idleConfig ) + { + return getTestableSupervisor( + id, + replicas, + taskCount, + useEarliestOffset, + false, + duration, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + suspended, + kafkaHost, + null, + null + ); + } + + private TestableKafkaSupervisor getTestableSupervisor( + @Nullable String id, + int replicas, + int taskCount, + boolean useEarliestOffset, + boolean resetOffsetAutomatically, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + String kafkaHost, + IdleConfig idleConfig, + Boolean usePerpetuallyRunningTasks + ) { final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); consumerProperties.put("myCustomKey", "myCustomValue"); @@ -5288,7 +5351,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, - null, + usePerpetuallyRunningTasks, taskStorage, taskMaster, indexerMetadataStorageCoordinator, From 160dd9f3c49363aac3283be0c5fe564b8d9f5b5c Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 12 Sep 2025 13:27:48 +0530 Subject: [PATCH 19/46] WIP: Fix the latch issue, use settable futures instead. --- .../indexing/KafkaClusterMetricsTest.java | 7 +- ...eekableStreamIndexTaskClientAsyncImpl.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 3 +- .../supervisor/SeekableStreamSupervisor.java | 91 ++++++++++++++----- 4 files changed, 73 insertions(+), 30 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index a209a8ca5d84..2c3ad9e97561 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -196,8 +196,8 @@ public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues .withLagCollectionIntervalMillis(100) .withLagCollectionRangeMillis(100) .withEnableTaskAutoScaler(true) - .withScaleActionPeriodMillis(60000) - .withScaleActionStartDelayMillis(100) + .withScaleActionPeriodMillis(5000) + .withScaleActionStartDelayMillis(10000) .withScaleOutThreshold(0) .withScaleInThreshold(10000) .withTriggerScaleOutFractionThreshold(0.001) @@ -206,7 +206,7 @@ public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues .withTaskCountMin(taskCount) .withScaleOutStep(1) .withScaleInStep(0) - .withMinTriggerScaleActionFrequencyMillis(100) + .withMinTriggerScaleActionFrequencyMillis(5000) .build(); final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( @@ -483,6 +483,7 @@ private KafkaSupervisorSpec createKafkaSupervisor( .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) .withTuningConfig(tuningConfig -> tuningConfig .withMaxRowsPerSegment(maxRowsPerSegment) + .withWorkerThreads(10) .withReleaseLocksOnHandoff(true)) .withIoConfig( ioConfig -> ioConfig diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 2d5d2b58cbd2..ee0f333b47c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -332,7 +332,7 @@ public ListenableFuture> pauseAndCheckp .handler(new BytesFullResponseHandler()) .onSuccess(r -> { if (r.getStatus().equals(HttpResponseStatus.OK)) { - log.info("Task [%s] paused successfully & Checkpoint requested succesffully", id); + log.info("Task [%s] paused successfully & Checkpoint requested successffully", id); return deserializeOffsetsMap(r.getContent()); } else if (r.getStatus().equals(HttpResponseStatus.ACCEPTED)) { // Return null, which triggers a loop later to wait for the task to enter PAUSED state. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index eff5e5c26b7b..2fea3d2a34cd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1803,6 +1803,7 @@ private void createNewSequenceFromIoConfig(SeekableStreamIndexTaskIOConfig offsetsFromTasks = pauseAndCheckpointAllTasks(); - final CountDownLatch currentLatch = checkpointsForConfigUpdateLatch; - if (currentLatch.await(CHECKPOINT_COMPLETION_LATCH_TIMEOUT, TimeUnit.SECONDS)) { - log.info("All tasks have been checkpointed successfully"); - } else { - log.warn("Timeout waiting for tasks to pause and checkpoint"); + if (offsetsFromTasks.isEmpty()) { + isDynamicAllocationOngoing.set(false); + return false; } + pendingConfigUpdateHook = () -> updateTaskConfigsAndCompleteAutoScaleEvent( + offsetsFromTasks, + scaleActionStopwatch, + desiredActiveTaskCount, + successfulScaleAutoScalerCallback + ); + return true; + } + + /** + * This function sends the config updates to all the tasks with new partition assignments and offsets. In an auto-scaling + * lifecycle, this marks the end of the scale action. + */ + private boolean updateTaskConfigsAndCompleteAutoScaleEvent( + Map offsetsFromTasks, + Stopwatch scaleActionStopwatch, + int desiredActiveTaskCount, + Runnable successfulScaleAutoScalerCallback + ) throws InterruptedException, ExecutionException + { changeTaskCountInIOConfig(desiredActiveTaskCount); Map> newPartitionGroups = calculateNewPartitionGroups(); @@ -648,11 +667,17 @@ private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount) } else { log.error("Failed to update task configurations for perpetual tasks"); } + isDynamicAllocationOngoing.set(false); emitAutoScalerRunMetric(scaleActionStopwatch); + + // You need to set the auto scaler specific stuff here. + successfulScaleAutoScalerCallback.run(); + dynamicTriggerLastRunTime = System.currentTimeMillis(); return success; } + private Map pauseAndCheckpointAllTasks() throws InterruptedException, ExecutionException { log.info("Pausing all tasks for perpetual task scaling"); @@ -665,10 +690,17 @@ private Map pauseAndCheckpointAllTasks() th } } + if (pauseFutures.isEmpty()) { + log.warn("No tasks found to pause for perpetual task scaling"); + return Collections.emptyMap(); + } + Map consolidatedLatestOffsets = new HashMap<>(); - checkpointsForConfigUpdateLatch = new CountDownLatch(pauseFutures.size()); - log.info("Staring a task checkpoint wait with count [%d]", pauseFutures.size()); + int tasksToWaitFor = pauseFutures.size(); + checkpointsToWaitFor = tasksToWaitFor; + log.info("Broadcasting pauseAndCheckpoint to [%d] tasks", tasksToWaitFor); + List>> latestOffsets = coalesceAndAwait(pauseFutures); for (Either> result : latestOffsets) { if (result.isError()) { @@ -687,6 +719,7 @@ private Map pauseAndCheckpointAllTasks() th )); } } + log.info("Successfully paused & checkpoints requested from [%d] tasks", pauseFutures.size()); return consolidatedLatestOffsets; } @@ -709,7 +742,6 @@ private Map> calculateNewPartitionGroups() for (PartitionIdType partition : allPartitions) { int taskGroupId = getTaskGroupIdForPartition(partition); newPartitionGroups.computeIfAbsent(taskGroupId, k -> new HashSet<>()).add(partition); - log.info("received taskGroupid [%d] and newPartitionGroup so far: %s", taskGroupId, newPartitionGroups); } log.info("Created [%d] new partition groups: %s", newPartitionGroups.size(), newPartitionGroups); @@ -731,6 +763,8 @@ private boolean sendConfigUpdatesToTasks(Map> newP Set partitions = entry.getValue(); TaskGroup existingTaskGroup = activelyReadingTaskGroups.get(taskGroupId); + log.info("Latest Committed Offsets from Metadata Storage: %s", latestCommittedOffsets); + log.info("Latest Committed Offsets from Task Pause: %s", latestTaskOffsetsOnPause); if (existingTaskGroup != null) { for (String taskId : existingTaskGroup.taskIds()) { SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig( @@ -1106,8 +1140,9 @@ public String getType() private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; protected final String dataSource; - private volatile CountDownLatch checkpointsForConfigUpdateLatch; private final AtomicBoolean isDynamicAllocationOngoing = new AtomicBoolean(false); + private int checkpointsToWaitFor = 0; + private Callable pendingConfigUpdateHook; private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; @@ -1969,6 +2004,10 @@ public TaskGroup addTaskGroupToPendingCompletionTaskGroup( @VisibleForTesting public void runInternal() { + if (isDynamicAllocationOngoing.get()) { + log.info("Skipping run because dynamic allocation is ongoing."); + return; + } try { possiblyRegisterListener(); @@ -3789,7 +3828,17 @@ public ListenableFuture> apply(List> apply(List stopTasksInGroup( @Nullable TaskGroup taskGroup, String stopReasonFormat, From 30330c541b1ce388e9a7146baa52992f5960e1de Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 12 Sep 2025 16:19:01 +0530 Subject: [PATCH 20/46] WIP: Additional logging --- .../seekablestream/SeekableStreamIndexTaskRunner.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 2fea3d2a34cd..083daaba91c6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1278,7 +1278,7 @@ private void addSequence(final SequenceMetadata Date: Fri, 12 Sep 2025 16:20:33 +0530 Subject: [PATCH 21/46] Fix build issue --- .../supervisor/LagBasedAutoScalerConfigBuilder.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java index fb10a4ad89e6..bc5ce28bb892 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java @@ -45,6 +45,7 @@ public class LagBasedAutoScalerConfigBuilder private boolean enableTaskAutoScaler; private long minTriggerScaleActionFrequencyMillis; private AggregateFunction lagAggregate; + private double stopTaskCountRatio; public LagBasedAutoScalerConfigBuilder withLagCollectionIntervalMillis(long lagCollectionIntervalMillis) { @@ -142,6 +143,12 @@ public LagBasedAutoScalerConfigBuilder withLagAggregate(AggregateFunction lagAgg return this; } + public LagBasedAutoScalerConfigBuilder withStopTaskCountRatio(double stopTaskCountRatio) + { + this.stopTaskCountRatio = stopTaskCountRatio; + return this; + } + public LagBasedAutoScalerConfig build() { return new LagBasedAutoScalerConfig( @@ -160,7 +167,8 @@ public LagBasedAutoScalerConfig build() scaleOutStep, enableTaskAutoScaler, minTriggerScaleActionFrequencyMillis, - lagAggregate + lagAggregate, + stopTaskCountRatio ); } } From c332e547d89cd84d01463852e3a674c863c04f1f Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Mon, 15 Sep 2025 14:10:02 +0530 Subject: [PATCH 22/46] Fix checkpoint and seeking issues --- .../indexing/KafkaClusterMetricsTest.java | 5 +- ...eekableStreamIndexTaskClientAsyncImpl.java | 5 +- .../SeekableStreamIndexTaskRunner.java | 51 ++++++++------ .../common/OrderedSequenceNumber.java | 14 +++- .../supervisor/SeekableStreamSupervisor.java | 68 ++++++++++++------- 5 files changed, 90 insertions(+), 53 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 2c3ad9e97561..29261e13d177 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -202,11 +202,12 @@ public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues .withScaleInThreshold(10000) .withTriggerScaleOutFractionThreshold(0.001) .withTriggerScaleInFractionThreshold(0.1) - .withTaskCountMax(10) + .withTaskCountMax(3) .withTaskCountMin(taskCount) .withScaleOutStep(1) .withScaleInStep(0) .withMinTriggerScaleActionFrequencyMillis(5000) + .withStopTaskCountRatio(1.0) .build(); final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( @@ -222,8 +223,6 @@ public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues cluster.callApi().postSupervisor(kafkaSupervisorSpec) ); - overlord.latchableEmitter().waitForEvent(event -> event.hasMetricName("task/autoScaler/requiredCount")); - overlord.latchableEmitter().waitForEvent( event -> event.hasMetricName("task/autoScaler/scaleActionTime") .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index ee0f333b47c1..f1a742b6fb6a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -217,7 +217,10 @@ public ListenableFuture updateConfigAsync(String taskId, TaskConfigUpda .jsonContent(jsonMapper, updateRequest); return makeRequest(taskId, requestBuilder) .handler(IgnoreHttpResponseHandler.INSTANCE) - .onSuccess(r -> true) + .onSuccess(r -> { + log.info("Successfully updated config for task [%s]", taskId); + return true; + }) .onHttpError(e -> { log.warn("Task [%s] config update failed due to http request failure [%s].", taskId, e.getMessage()); return Either.value(false); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 083daaba91c6..ac70a484be8d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -208,6 +208,8 @@ public enum Status protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); private final SeekableStreamIndexTask task; + private volatile Set> assignment; + private volatile RecordSupplier recordSupplier; private SeekableStreamIndexTaskIOConfig ioConfig; private final SeekableStreamIndexTaskTuningConfig tuningConfig; private final InputRowSchema inputRowSchema; @@ -253,7 +255,7 @@ public enum Status private volatile DateTime minMessageTime; private volatile DateTime maxMessageTime; private final ScheduledExecutorService rejectionPeriodUpdaterExec; - private volatile boolean waitForConfigUpdate = false; + private AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, @@ -458,6 +460,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier(toolbox)) { + this.recordSupplier = recordSupplier; if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); @@ -602,15 +605,16 @@ public void run() // restart publishing of sequences (if any) maybePersistAndPublishSequences(committerSupplier); - Set> assignment = assignPartitions(recordSupplier); - possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); - seekToStartingSequence(recordSupplier, assignment); + assignment = assignPartitions(this.recordSupplier); + possiblyResetDataSourceMetadata(toolbox, this.recordSupplier, assignment); + seekToStartingSequence(this.recordSupplier, assignment); ingestionState = IngestionState.BUILD_SEGMENTS; // Main loop. // Could eventually support leader/follower mode (for keeping replicas more in sync) - boolean stillReading = !assignment.isEmpty(); + log.info("Task perpetuallyRunning: %s", task.isPerpetuallyRunning()); + boolean stillReading = !assignment.isEmpty() || task.isPerpetuallyRunning(); status = Status.READING; Throwable caughtExceptionInner = null; @@ -620,8 +624,8 @@ public void run() // The partition assignments may have changed while paused by a call to setEndOffsets() so reassign // partitions upon resuming. Don't call "seekToStartingSequence" after "assignPartitions", because there's // no need to re-seek here. All we're going to be doing is dropping partitions. - assignment = assignPartitions(recordSupplier); - possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); + assignment = assignPartitions(this.recordSupplier); + possiblyResetDataSourceMetadata(toolbox, this.recordSupplier, assignment); if (assignment.isEmpty()) { log.debug("All partitions have been fully read."); @@ -650,12 +654,12 @@ public void run() // calling getRecord() ensures that exceptions specific to kafka/kinesis like OffsetOutOfRangeException // are handled in the subclasses. List> records = getRecords( - recordSupplier, + this.recordSupplier, toolbox ); // note: getRecords() also updates assignment - stillReading = !assignment.isEmpty(); + stillReading = !assignment.isEmpty() || task.isPerpetuallyRunning(); SequenceMetadata sequenceToCheckpoint = null; AppenderatorDriverAddResult pushTriggeringAddResult = null; @@ -759,8 +763,8 @@ public void onFailure(Throwable t) if (!moreToReadAfterThisRecord && assignment.remove(record.getStreamPartition())) { log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); - recordSupplier.assign(assignment); - stillReading = !assignment.isEmpty(); + this.recordSupplier.assign(assignment); + stillReading = !assignment.isEmpty() || task.isPerpetuallyRunning(); } } @@ -1737,7 +1741,7 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt throws InterruptedException { authorizationCheck(req); - if (!waitForConfigUpdate) { + if (!waitForConfigUpdate.get()) { return Response.status(409).entity("Task must be paused for checkpoint completion before updating config").build(); } try { @@ -1748,6 +1752,10 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt setIOConfig(newIoConfig); createNewSequenceFromIoConfig(newIoConfig); + assignment = assignPartitions(recordSupplier); + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); + seekToStartingSequence(recordSupplier, assignment); + log.info("Config updated to [%s]", this.ioConfig); toolbox.getEmitter().emit(ServiceMetricEvent.builder() .setDimension(DruidMetrics.TASK_ID, task.getId()) @@ -1756,19 +1764,19 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt .setMetric("task/config/update/success", 1) .build(ImmutableMap.of())); resume(); - waitForConfigUpdate = false; + waitForConfigUpdate.set(false); return Response.ok().build(); } catch (Exception e) { log.makeAlert(e, "Failed to update task config"); - waitForConfigUpdate = false; + waitForConfigUpdate.set(false); return Response.serverError().entity(e.getMessage()).build(); } } private void setIOConfig( SeekableStreamIndexTaskIOConfig ioConfig - ) throws IOException + ) { this.ioConfig = ioConfig; this.stream = ioConfig.getStartSequenceNumbers().getStream(); @@ -1867,7 +1875,7 @@ public Response setEndOffsets( } else { try { // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock - if (!waitForConfigUpdate) { + if (!waitForConfigUpdate.get()) { pauseLock.lockInterruptibly(); } // Perform all sequence related checks before checking for isPaused() @@ -1893,7 +1901,7 @@ public Response setEndOffsets( || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resetNextCheckpointTime(); - if (!waitForConfigUpdate) { + if (!waitForConfigUpdate.get()) { resume(); } return Response.ok(sequenceNumbers).build(); @@ -1931,7 +1939,7 @@ public Response setEndOffsets( log.info("Latest sequence after setting end offsets: [%s]", latestSequence); // if this is the final checkpoint, or if the task is paused for checkpoint completion and updateConfig is supposed to // finish the current sequence, we just update the end offsets of the latest sequence. - if (finish || waitForConfigUpdate) { + if (finish || waitForConfigUpdate.get()) { log.info( "Sequence[%s] end offsets updated from [%s] to [%s].", latestSequence.getSequenceName(), @@ -1972,13 +1980,13 @@ public Response setEndOffsets( .build(); } finally { - if (!waitForConfigUpdate) { + if (!waitForConfigUpdate.get()) { pauseLock.unlock(); } } } - if (!waitForConfigUpdate) { + if (!waitForConfigUpdate.get()) { resume(); } @@ -2042,11 +2050,10 @@ public Response pauseAndCheckpointHTTP( ) throws InterruptedException, JsonProcessingException { authorizationCheck(req); - if (waitForConfigUpdate) { + if (!waitForConfigUpdate.compareAndSet(false, true)) { return Response.ok().entity("Task is already paused for checkpoint completion").build(); } pause(); - waitForConfigUpdate = true; checkpointSequences(); return Response.ok().entity(toolbox.getJsonMapper().writeValueAsString(getCurrentOffsets())).build(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java index cbc61edbe6ee..089a9db3981a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java @@ -98,7 +98,17 @@ public boolean isAvailableWithEarliest(OrderedSequenceNumber public boolean isMoreToReadBeforeReadingRecord(OrderedSequenceNumber end, boolean isEndOffsetExclusive) { - final int compareToEnd = this.compareTo(end); - return isEndOffsetExclusive ? compareToEnd < 0 : compareToEnd <= 0; + // This happens in the situations where earlier sequences had a different partition mapping and has now been updated. + // Since the end is not defined, we can't really say if there is more to read or not. + try { + if (end.sequenceNumber == null) { + return false; + } + final int compareToEnd = this.compareTo(end); + return isEndOffsetExclusive ? compareToEnd < 0 : compareToEnd <= 0; + } catch (Exception e) { + // In case of any exception, we return false to avoid reading unwanted data. + return false; + } } } 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 633a2ddf309a..44c68eb94c4b 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 @@ -758,30 +758,24 @@ private boolean sendConfigUpdatesToTasks(Map> newP log.info("Sending configuration updates to the following partition groups %s", newPartitionGroups); List> updateFutures = new ArrayList<>(); Map latestCommittedOffsets = getOffsetsFromMetadataStorage(); - for (Map.Entry> entry : newPartitionGroups.entrySet()) { + for (Map.Entry entry: activelyReadingTaskGroups.entrySet()) { int taskGroupId = entry.getKey(); - Set partitions = entry.getValue(); - - TaskGroup existingTaskGroup = activelyReadingTaskGroups.get(taskGroupId); - log.info("Latest Committed Offsets from Metadata Storage: %s", latestCommittedOffsets); - log.info("Latest Committed Offsets from Task Pause: %s", latestTaskOffsetsOnPause); - if (existingTaskGroup != null) { - for (String taskId : existingTaskGroup.taskIds()) { - SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig( - partitions, - existingTaskGroup, - latestCommittedOffsets, - latestTaskOffsetsOnPause - ); - TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); - - log.debug("Updating config for task [%s] with partitions [%s]", taskId, partitions); - updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); - } + TaskGroup existingTaskGroup = entry.getValue(); + + Set partitionsForThisTask = new HashSet<>(newPartitionGroups.get(taskGroupId)); + SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig( + partitionsForThisTask, + existingTaskGroup, + latestCommittedOffsets, + latestTaskOffsetsOnPause + ); + TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); + for (String taskId : existingTaskGroup.taskIds()) { + log.info("Updating config for task [%s] with partitions [%s]", taskId, partitionsForThisTask); + updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); } } - if (updateFutures.isEmpty()) { log.info("No configuration updates needed"); return true; @@ -1051,13 +1045,12 @@ public void handle() throws ExecutionException, InterruptedException // check validity of previousCheckpoint int index = checkpoints.size(); + // latest checkpoints are being compared first. for (int sequenceId : checkpoints.descendingKeySet()) { Map checkpoint = checkpoints.get(sequenceId); // We have already verified the stream of the current checkpoint is same with that in ioConfig. // See checkpoint(). - if (checkpoint.equals(checkpointMetadata.getSeekableStreamSequenceNumbers() - .getPartitionSequenceNumberMap() - )) { + if (isCheckpointSignatureValid(checkpoint, checkpointMetadata)) { break; } index--; @@ -1111,6 +1104,31 @@ public String getType() } } + /** + * This function verifies whether the checkpoint received from a task is valid or not. + * In non-perpetual tasks, the checkpoint is valid if it matches one of the partition <> sequence numbers in the task checkpoints. + * In perpetual tasks, this does not hold valid anymore because partition assignments can change during dynamic scaling. We'll compare + * it with supervisor level partition group's partition assignments now. + */ + private boolean isCheckpointSignatureValid( + Map checkpoint, + SeekableStreamDataSourceMetadata checkpointMetadata + ) + { + var checkpointPresentAlready = checkpoint; + var checkpointProposed = checkpointMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap(); + // TODO: Think about what we can do here for now. + if (spec.usePerpetuallyRunningTasks()) { +// Set assignedPartitions = partitionGroups.getOrDefault( +// checkpointMetadata.getTaskGroupId(), +// Collections.emptySet() +// ); + return true; + } + + return checkpoint.equals(checkpointMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap()); + } + // Map<{group id}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); @@ -3832,9 +3850,9 @@ public ListenableFuture> apply(List Date: Mon, 15 Sep 2025 18:14:02 +0530 Subject: [PATCH 23/46] Fix perpetually running task config send --- .../druid/indexing/seekablestream/SeekableStreamIndexTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 75aa507d640b..eb9dafde626a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -156,7 +156,7 @@ public SeekableStreamIndexTaskIOConfig getI return ioConfig; } - @JsonProperty + @JsonProperty("isPerpetuallyRunning") public boolean isPerpetuallyRunning() { return isPerpetuallyRunning; From 653ef632c1d19d3118947df5a9f127400d0496cd Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Tue, 16 Sep 2025 11:08:50 +0530 Subject: [PATCH 24/46] WIP: Continous publish and handoffs for perpetual tasks --- .../SeekableStreamIndexTaskRunner.java | 193 ++++++++++++------ 1 file changed, 131 insertions(+), 62 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index ac70a484be8d..4c1ebf830b56 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -77,6 +77,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.PendingSegmentRecord; @@ -130,6 +131,7 @@ import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -149,7 +151,6 @@ public abstract class SeekableStreamIndexTaskRunner publishingTask; + private final long PUBLISH_INTERVAL_MS = 5000; + private Supplier committerSupplier; + public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, @@ -292,6 +297,10 @@ public SeekableStreamIndexTaskRunner( ); } resetNextCheckpointTime(); + this.publishingExec = ScheduledExecutors.fixed( + 1, + StringUtils.encodeForFormat("Publisher-" + task.getId()) + "-%d" + ); } public TaskStatus run(TaskToolbox toolbox) @@ -457,6 +466,9 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception //milliseconds waited for created segments to be handed off long handoffWaitMs = 0L; + if (task.isPerpetuallyRunning()) { + startContinuousPublishing(); + } try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier(toolbox)) { @@ -581,7 +593,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception } // Set up committer. - final Supplier committerSupplier = () -> { + committerSupplier = () -> { final Map snapshot = ImmutableMap.copyOf(currOffsets); lastPersistedOffsets.clear(); lastPersistedOffsets.putAll(snapshot); @@ -603,7 +615,9 @@ public void run() }; // restart publishing of sequences (if any) - maybePersistAndPublishSequences(committerSupplier); + if (!task.isPerpetuallyRunning()) { + maybePersistAndPublishSequences(committerSupplier); + } assignment = assignPartitions(this.recordSupplier); possiblyResetDataSourceMetadata(toolbox, this.recordSupplier, assignment); @@ -628,13 +642,17 @@ public void run() possiblyResetDataSourceMetadata(toolbox, this.recordSupplier, assignment); if (assignment.isEmpty()) { - log.debug("All partitions have been fully read."); - publishOnStop.set(true); - stopRequested.set(true); + if (!task.isPerpetuallyRunning()) { + log.debug("All partitions have been fully read."); + publishOnStop.set(true); + stopRequested.set(true); + } } } // if stop is requested or task's end sequence is set by call to setEndOffsets method with finish set to true + // TODO: When the last sequence metadata is checkpointed, do we really want to transition to PUBLISHING for perpetually running tasks? + // I understand first 2 points of PUBLISHING state, a stop was requested, sequences are finished, but what has a last sequence metadata being checkpointed got to do with it? if (stopRequested.get() || sequences.size() == 0 || getLastSequenceMetadata().isCheckpointed()) { status = Status.PUBLISHING; } @@ -855,66 +873,15 @@ public void onFailure(Throwable t) // We need to copy sequences here, because the success callback in publishAndRegisterHandoff removes items from // the sequence list. If a publish finishes before we finish iterating through the sequence list, we can // end up skipping some sequences. - List> sequencesSnapshot = new ArrayList<>(sequences); - for (int i = 0; i < sequencesSnapshot.size(); i++) { - final SequenceMetadata sequenceMetadata = sequencesSnapshot.get(i); - if (!publishingSequences.contains(sequenceMetadata.getSequenceName()) - && !publishedSequences.contains(sequenceMetadata.getSequenceName())) { - final boolean isLast = i == (sequencesSnapshot.size() - 1); - if (isLast) { - // Shorten endOffsets of the last sequence to match currOffsets. - sequenceMetadata.setEndOffsets(currOffsets); - } - - // Update assignments of the sequence, which should clear them. (This will be checked later, when the - // Committer is built.) - sequenceMetadata.updateAssignments(currOffsets, this::isMoreToReadAfterReadingRecord); - publishingSequences.add(sequenceMetadata.getSequenceName()); - // persist already done in finally, so directly add to publishQueue - publishAndRegisterHandoff(sequenceMetadata); - } - } + populateSequencesToPublish(); if (backgroundThreadException != null) { throw new RuntimeException(backgroundThreadException); } - // Wait for publish futures to complete. - Futures.allAsList(publishWaitList).get(); - - // Wait for handoff futures to complete. - // Note that every publishing task (created by calling AppenderatorDriver.publish()) has a corresponding - // handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it - // failed to persist sequences. It might also return null if handoff failed, but was recoverable. - // See publishAndRegisterHandoff() for details. - List handedOffList = Collections.emptyList(); - ingestionState = IngestionState.SEGMENT_AVAILABILITY_WAIT; - if (tuningConfig.getHandoffConditionTimeout() == 0) { - handedOffList = Futures.allAsList(handOffWaitList).get(); - } else { - final long start = System.nanoTime(); - try { - handedOffList = Futures.allAsList(handOffWaitList) - .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); - } - catch (TimeoutException e) { - // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception - // here. - log.makeAlert("Timeout waiting for handoff") - .addData("taskId", task.getId()) - .addData("handoffConditionTimeout", tuningConfig.getHandoffConditionTimeout()) - .emit(); - } - finally { - handoffWaitMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); - } - } - - for (SegmentsAndCommitMetadata handedOff : handedOffList) { - log.info( - "Handoff complete for segments: %s", - String.join(", ", Lists.transform(handedOff.getSegments(), DataSegment::toString)) - ); + // Wait for publish futures to complete if it's a standard task. + if (!task.isPerpetuallyRunning()) { + handoffWaitMs = waitForPublishAndHandoffCompletion(); } appenderator.close(); @@ -988,6 +955,102 @@ public void onFailure(Throwable t) return TaskStatus.success(task.getId()); } + private long waitForPublishAndHandoffCompletion() throws ExecutionException, InterruptedException + { + long handoffWaitMs = 0L; + Futures.allAsList(publishWaitList).get(); + + // Wait for handoff futures to complete. + // Note that every publishing task (created by calling AppenderatorDriver.publish()) has a corresponding + // handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it + // failed to persist sequences. It might also return null if handoff failed, but was recoverable. + // See publishAndRegisterHandoff() for details. + List handedOffList = Collections.emptyList(); + ingestionState = IngestionState.SEGMENT_AVAILABILITY_WAIT; + if (tuningConfig.getHandoffConditionTimeout() == 0) { + handedOffList = Futures.allAsList(handOffWaitList).get(); + } else { + final long start = System.nanoTime(); + try { + handedOffList = Futures.allAsList(handOffWaitList) + .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + } + catch (TimeoutException e) { + // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception + // here. + log.makeAlert("Timeout waiting for handoff") + .addData("taskId", task.getId()) + .addData("handoffConditionTimeout", tuningConfig.getHandoffConditionTimeout()) + .emit(); + } + finally { + handoffWaitMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + } + } + + for (SegmentsAndCommitMetadata handedOff : handedOffList) { + log.info( + "Handoff complete for segments: %s", + String.join(", ", Lists.transform(handedOff.getSegments(), DataSegment::toString)) + ); + } + return handoffWaitMs; + } + + private void populateSequencesToPublish() + { + List> sequencesSnapshot = new ArrayList<>(sequences); + for (int i = 0; i < sequencesSnapshot.size(); i++) { + final SequenceMetadata sequenceMetadata = sequencesSnapshot.get(i); + if (!publishingSequences.contains(sequenceMetadata.getSequenceName()) + && !publishedSequences.contains(sequenceMetadata.getSequenceName())) { + final boolean isLast = i == (sequencesSnapshot.size() - 1); + if (isLast) { + // Shorten endOffsets of the last sequence to match currOffsets. + sequenceMetadata.setEndOffsets(currOffsets); + } + + // Update assignments of the sequence, which should clear them. (This will be checked later, when the + // Committer is built.) + sequenceMetadata.updateAssignments(currOffsets, this::isMoreToReadAfterReadingRecord); + publishingSequences.add(sequenceMetadata.getSequenceName()); + // persist already done in finally, so directly add to publishQueue + publishAndRegisterHandoff(sequenceMetadata); + } + } + } + + /** + * Start the continuous publishing executor for perpetual tasks + * Call this after the main reading loop starts + */ + private void startContinuousPublishing() + { + log.info("Starting continuous publishing for perpetual task [%s]", task.getId()); + + publishingTask = publishingExec.scheduleAtFixedRate( + this::performContinuousPublishing, + PUBLISH_INTERVAL_MS, + PUBLISH_INTERVAL_MS, + TimeUnit.MILLISECONDS + ); + } + + /** + * The method that runs periodically to check if there are sequences to be published + * for perpetual tasks + */ + private void performContinuousPublishing() + { + try { + maybePersistAndPublishSequences(committerSupplier); + waitForPublishAndHandoffCompletion(); + } + catch (Exception e) { + log.error(e, "Encountered exception while publishing in background thread"); + } + } + private TaskLockType getTaskLockType() { return TaskLocks.determineLockTypeForAppend(task.getContext()); @@ -1507,6 +1570,9 @@ public void stopForcefully() if (runThread != null) { runThread.interrupt(); } + if (publishingExec != null) { + publishingExec.shutdownNow(); + } } public void stopGracefully() @@ -1553,6 +1619,9 @@ public void stopGracefully() catch (Exception e) { throw new RuntimeException(e); } + if (publishingExec != null) { + publishingExec.shutdown(); + } } @POST From a6c949cf8e17337f4daf3d1e3f3f92bc18891a2b Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Tue, 16 Sep 2025 14:01:12 +0530 Subject: [PATCH 25/46] WIP: Save updated task config in taskStorage, todo: save it via taksQueue instead --- .../overlord/HeapMemoryTaskStorage.java | 11 +++++ .../overlord/MetadataTaskStorage.java | 23 ++++++++++ .../druid/indexing/overlord/TaskStorage.java | 7 ++++ .../SeekableStreamIndexTask.java | 5 +++ .../SeekableStreamIndexTaskRunner.java | 3 +- .../supervisor/SeekableStreamSupervisor.java | 31 +++++++++++++- .../MetadataStorageActionHandler.java | 8 ++++ .../SQLMetadataStorageActionHandler.java | 42 +++++++++++++++++++ ...SeekableStreamIndexTaskRunnerAuthTest.java | 6 +++ .../SeekableStreamSupervisorStateTest.java | 6 +++ .../MetadataStorageActionHandlerTest.java | 6 +++ .../org/apache/druid/cli/CliPeonTest.java | 6 +++ 12 files changed, 151 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index b3201ea9314f..fdee190d545b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -304,6 +304,17 @@ public List getLocks(final String taskid) } } + @Override + public void updateTask(Task task) + { + final Optional status = getStatus(task.getId()); + if (!status.isPresent()) { + throw new IllegalStateException("No task found for id [" + task.getId() + "]"); + } + final TaskInfo updatedTaskInfo = new TaskInfo(DateTimes.nowUtc(), status.get(), task); + tasks.put(task.getId(), updatedTaskInfo); + } + @Override public void removeTasksOlderThan(final long timestamp) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 9dba47c62780..55e8215876c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -296,4 +296,27 @@ private Map getLocksWithIds(final String taskid) { return handler.getLocks(taskid); } + + @Override + public void updateTask(Task task) + { + Preconditions.checkNotNull(task, "task"); + log.info("Updating task [%s].", task.getId()); + final String taskId = task.getId(); + + Optional existingTask = getTask(taskId); + if (!existingTask.isPresent()) { + throw new ISE("No task found for id [%s]", taskId); + } + + try { + handler.update(taskId, task); + } + catch (DruidException e) { + throw e; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index 49a248ec4b8b..f1b9c92cedfe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -182,4 +182,11 @@ default List getTaskInfos( * @return list of TaskLocks for the given task */ List getLocks(String taskid); + + /** + * Updates an existing task with new configuration. + * This is used for updating task payload (e.g., IOConfig changes for perpetual tasks) + * while preserving the task status and other metadata. + */ + void updateTask(Task task); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index eb9dafde626a..f06177f5d2e2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -39,6 +39,7 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.PendingSegmentAllocatingTask; +import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; @@ -318,4 +319,8 @@ public Appenderator getAppenderator() { return runnerSupplier.get(); } + + public abstract SeekableStreamIndexTask withNewIoConfig( + SeekableStreamIndexTaskIOConfig newIoConfig + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 4c1ebf830b56..bcb37527ebc1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -258,7 +258,7 @@ public enum Status private AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); private ScheduledExecutorService publishingExec; private ScheduledFuture publishingTask; - private final long PUBLISH_INTERVAL_MS = 5000; + private final long PUBLISH_INTERVAL_MS = 2000; private Supplier committerSupplier; @@ -466,6 +466,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception //milliseconds waited for created segments to be handed off long handoffWaitMs = 0L; + log.info("Task perpetually running: %s", task.isPerpetuallyRunning()); if (task.isPerpetuallyRunning()) { startContinuousPublishing(); } 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 44c68eb94c4b..f2d8ec230f88 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 @@ -769,10 +769,9 @@ private boolean sendConfigUpdatesToTasks(Map> newP latestCommittedOffsets, latestTaskOffsetsOnPause ); - TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); for (String taskId : existingTaskGroup.taskIds()) { log.info("Updating config for task [%s] with partitions [%s]", taskId, partitionsForThisTask); - updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); + updateFutures.add(persistThenUpdateTaskConfig(taskId, newIoConfig)); } } @@ -800,6 +799,34 @@ private boolean sendConfigUpdatesToTasks(Map> newP return allSucceeded; } + private ListenableFuture persistThenUpdateTaskConfig( + String taskId, + SeekableStreamIndexTaskIOConfig newIoConfig + ) + { + return Futures.transformAsync( + workerExec.submit(() -> { + Optional existingTaskOpt = taskStorage.getTask(taskId); + if (!existingTaskOpt.isPresent()) { + throw new ISE("Task [%s] not found in storage", taskId); + } + SeekableStreamIndexTask existingTask = + (SeekableStreamIndexTask) existingTaskOpt.get(); + SeekableStreamIndexTask updatedTask = existingTask.withNewIoConfig(newIoConfig); + log.info("Persisting updated config for task [%s] to storage", taskId); + taskStorage.updateTask(updatedTask); + return updatedTask; + }), + (persistedTask) -> { + log.info("Sending config update to running task [%s]", taskId); + TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); + return taskClient.updateConfigAsync(taskId, updateRequest); + }, + workerExec + ); + } + + /** * Handles obsolete task groups when scaling down. * Pauses tasks in task groups that are no longer needed and removes them from activelyReadingTaskGroups. diff --git a/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 23e315aac0c6..83259bee7d8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -189,4 +189,12 @@ default List getTaskInfos( */ void populateTaskTypeAndGroupIdAsync(); + + /** + * Update the task entry with the given id. + */ + void update( + final String id, + final @NotNull Task entry + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index b7190edf6518..9ce437912f38 100644 --- a/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -146,6 +146,26 @@ public void insert( } } + @Override + public void update( + final String id, + final @NotNull Task entry + ) + { + try { + getConnector().retryWithHandle( + handle -> updateEntryWithHandle(handle, id, entry), + this::isTransientDruidException + ); + } + catch (CallbackFailedException e) { + propagateAsRuntimeException(e.getCause()); + } + catch (Exception e) { + propagateAsRuntimeException(e); + } + } + private void propagateAsRuntimeException(Throwable t) { Throwables.propagateIfPossible(t); @@ -192,6 +212,28 @@ private Void insertEntryWithHandle( } } + private Void updateEntryWithHandle( + Handle handle, + String entryId, + Object entry + ) + { + try { + final String sql = StringUtils.format( + "UPDATE %s SET payload = :payload WHERE id = :id", + getEntryTable() + ); + handle.createStatement(sql) + .bind("id", entryId) + .bind("payload", jsonMapper.writeValueAsBytes(entry)) + .execute(); + return null; + } + catch (Throwable t) { + throw wrapInDruidException(entryId, t); + } + } + private boolean isTransientDruidException(Throwable t) { if (t instanceof CallbackFailedException) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index 7c319d8887aa..d9671e20239e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -395,6 +395,12 @@ protected RecordSupplier newTaskRecordSupplier(final { return null; } + + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return null; + } } private static class TestSeekableStreamIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig 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 11a4a7564957..d1ddedc46ed7 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 @@ -2974,6 +2974,12 @@ protected RecordSupplier newTaskRecordSupplier(final return recordSupplier; } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return this; + } + @Override public String getType() { diff --git a/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java b/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java index 88db9e86f7ea..589e6b4bac34 100644 --- a/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java @@ -148,6 +148,12 @@ public void populateTaskTypeAndGroupIdAsync() { } + + @Override + public void update(String id, Task entry) + { + + } }; } } 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 e6fdd8d772ec..a1428d535888 100644 --- a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java +++ b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java @@ -281,6 +281,12 @@ protected RecordSupplier newTaskRecordSupplier(final return null; } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return this; + } + @Override public String getCurrentRunnerStatus() { From 2e66e4db41f1634b29f6ec4788e82d9cf51b3ab0 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Tue, 16 Sep 2025 19:06:54 +0530 Subject: [PATCH 26/46] Fix succeeding tasks issues --- .../rabbitstream/RabbitStreamIndexTask.java | 16 ++++++++ .../druid/indexing/kafka/KafkaIndexTask.java | 17 ++++++++ .../indexing/kinesis/KinesisIndexTask.java | 17 ++++++++ .../druid/indexing/overlord/TaskQueue.java | 40 +++++++++++++++++++ .../supervisor/SeekableStreamSupervisor.java | 26 +++++++++++- 5 files changed, 114 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java index b742fef389b9..85bf39cef1c1 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.utils.RuntimeInfo; @@ -123,6 +124,21 @@ protected RabbitStreamRecordSupplier newTaskRecordSupplier() ); } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return new RabbitStreamIndexTask( + getId(), + getSupervisorId(), + getTaskResource(), + getDataSchema(), + getTuningConfig(), + (RabbitStreamIndexTaskIOConfig) newIoConfig, + getContext(), + configMapper + ); + } + @Override @JsonProperty public RabbitStreamIndexTaskTuningConfig getTuningConfig() diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index e8001ba5fc2d..c9904239b833 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.security.AuthorizationUtils; @@ -134,6 +135,22 @@ protected KafkaRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) } } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return new KafkaIndexTask( + getId(), + getSupervisorId(), + getTaskResource(), + getDataSchema(), + getTuningConfig(), + (KafkaIndexTaskIOConfig) newIoConfig, + getContext(), + isPerpetuallyRunning(), + configMapper + ); + } + @Override @JsonProperty public KafkaIndexTaskTuningConfig getTuningConfig() diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index 3b8e7438ad76..d7ff577f03b7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.DataSchema; @@ -154,6 +155,22 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) ); } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return new KinesisIndexTask( + getId(), + getSupervisorId(), + getTaskResource(), + getDataSchema(), + getTuningConfig(), + (KinesisIndexTaskIOConfig) newIoConfig, + getContext(), + useListShards, + awsCredentialsConfig + ); + } + @Override @JsonProperty public KinesisIndexTaskTuningConfig getTuningConfig() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 0cfe481a8fea..1a01ae1e118a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -555,6 +555,46 @@ public boolean add(final Task task) } } + public void update(final Task task) + { + IdUtils.validateId("Task ID", task.getId()); + + if (!taskStorage.getTask(task.getId()).isPresent()) { + throw new ISE("Task[%s] does not exist in storage", task.getId()); + } + + validateTaskPayload(task); + startStopLock.readLock().lock(); + + try { + Preconditions.checkState(active, "Queue is not active!"); + Preconditions.checkNotNull(task, "task"); + + final String taskId = task.getId(); + if (!activeTasks.containsKey(taskId)) { + throw new ISE("Task[%s] is not in the queue", taskId); + } + + updateTaskEntry(taskId, entry -> { + // Create new TaskInfo with updated task while preserving created time and status + final TaskInfo existingTaskInfo = entry.taskInfo; + entry.taskInfo = new TaskInfo( + existingTaskInfo.getCreatedTime(), + existingTaskInfo.getStatus(), + task + ); + log.info("Updated task [%s] in queue hashmap", taskId); + }); + + taskStorage.updateTask(task); + log.info("Updated task [%s] in storage", taskId); + requestManagement(); + } + finally { + startStopLock.readLock().unlock(); + } + } + @GuardedBy("startStopLock") private void addTaskInternal(final TaskInfo taskInfo, final DateTime updateTime) { 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 f2d8ec230f88..096a0c2fd376 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 @@ -197,7 +197,8 @@ public class TaskGroup // this task group has completed successfully, at which point this will be destroyed and a new task group will be // created with new starting sequences. This allows us to create replacement tasks for failed tasks that process the // same sequences, even if the values in [partitionGroups] has been changed. - final ImmutableMap startingSequences; + // In perpetually-running tasks mode, the actively running task groups will be replaced with new task groups with updated starting sequences. + ImmutableMap startingSequences; // We don't include closed partitions in the starting offsets. However, we keep the full unfiltered map of // partitions, only used for generating the sequence name, to avoid ambiguity in sequence names if mulitple @@ -317,6 +318,12 @@ public String toString() ", tasks=" + tasks + '}'; } + + public TaskGroup withStartingSequences(Map newStartingSequences) + { + this.startingSequences = ImmutableMap.copyOf(newStartingSequences); + return this; + } } private class TaskData @@ -632,6 +639,7 @@ private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount, activelyReadingTaskGroups.size(), desiredActiveTaskCount); Map offsetsFromTasks = pauseAndCheckpointAllTasks(); + if (offsetsFromTasks.isEmpty()) { isDynamicAllocationOngoing.set(false); return false; @@ -663,6 +671,7 @@ private boolean updateTaskConfigsAndCompleteAutoScaleEvent( if (success) { updatePartitionGroupsForPerpetualTasks(newPartitionGroups); + log.info("Successfully updated task configurations for perpetual tasks scaling"); } else { log.error("Failed to update task configurations for perpetual tasks"); @@ -769,10 +778,13 @@ private boolean sendConfigUpdatesToTasks(Map> newP latestCommittedOffsets, latestTaskOffsetsOnPause ); + Map newStartingSequences = newIoConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap(); + TaskGroup newTaskGroup = existingTaskGroup.withStartingSequences(newStartingSequences); for (String taskId : existingTaskGroup.taskIds()) { log.info("Updating config for task [%s] with partitions [%s]", taskId, partitionsForThisTask); updateFutures.add(persistThenUpdateTaskConfig(taskId, newIoConfig)); } + activelyReadingTaskGroups.put(taskGroupId, newTaskGroup); } if (updateFutures.isEmpty()) { @@ -814,7 +826,7 @@ private ListenableFuture persistThenUpdateTaskConfig( (SeekableStreamIndexTask) existingTaskOpt.get(); SeekableStreamIndexTask updatedTask = existingTask.withNewIoConfig(newIoConfig); log.info("Persisting updated config for task [%s] to storage", taskId); - taskStorage.updateTask(updatedTask); + updateTaskIoConfigInQueueOrStorage(updatedTask); return updatedTask; }), (persistedTask) -> { @@ -826,6 +838,16 @@ private ListenableFuture persistThenUpdateTaskConfig( ); } + private void updateTaskIoConfigInQueueOrStorage(SeekableStreamIndexTask updatedTask) + { + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + taskQueue.get().update(updatedTask); + } else { + taskStorage.updateTask(updatedTask); + } + } + /** * Handles obsolete task groups when scaling down. From b3a1cf60deeb54ab21e2a6f0c7a816ebf976ee30 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 17 Sep 2025 13:20:23 +0530 Subject: [PATCH 27/46] revert separate thread for scheduling publishes --- .../SeekableStreamIndexTaskRunner.java | 59 +------------------ 1 file changed, 2 insertions(+), 57 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index bcb37527ebc1..2140403dd96a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -77,7 +77,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.PendingSegmentRecord; @@ -131,7 +130,6 @@ import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -256,10 +254,6 @@ public enum Status private volatile DateTime maxMessageTime; private final ScheduledExecutorService rejectionPeriodUpdaterExec; private AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); - private ScheduledExecutorService publishingExec; - private ScheduledFuture publishingTask; - private final long PUBLISH_INTERVAL_MS = 2000; - private Supplier committerSupplier; public SeekableStreamIndexTaskRunner( @@ -297,10 +291,6 @@ public SeekableStreamIndexTaskRunner( ); } resetNextCheckpointTime(); - this.publishingExec = ScheduledExecutors.fixed( - 1, - StringUtils.encodeForFormat("Publisher-" + task.getId()) + "-%d" - ); } public TaskStatus run(TaskToolbox toolbox) @@ -467,10 +457,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception //milliseconds waited for created segments to be handed off long handoffWaitMs = 0L; log.info("Task perpetually running: %s", task.isPerpetuallyRunning()); - if (task.isPerpetuallyRunning()) { - startContinuousPublishing(); - } - try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier(toolbox)) { this.recordSupplier = recordSupplier; @@ -594,7 +580,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception } // Set up committer. - committerSupplier = () -> { + final Supplier committerSupplier = () -> { final Map snapshot = ImmutableMap.copyOf(currOffsets); lastPersistedOffsets.clear(); lastPersistedOffsets.putAll(snapshot); @@ -616,9 +602,7 @@ public void run() }; // restart publishing of sequences (if any) - if (!task.isPerpetuallyRunning()) { - maybePersistAndPublishSequences(committerSupplier); - } + maybePersistAndPublishSequences(committerSupplier); assignment = assignPartitions(this.recordSupplier); possiblyResetDataSourceMetadata(toolbox, this.recordSupplier, assignment); @@ -652,8 +636,6 @@ public void run() } // if stop is requested or task's end sequence is set by call to setEndOffsets method with finish set to true - // TODO: When the last sequence metadata is checkpointed, do we really want to transition to PUBLISHING for perpetually running tasks? - // I understand first 2 points of PUBLISHING state, a stop was requested, sequences are finished, but what has a last sequence metadata being checkpointed got to do with it? if (stopRequested.get() || sequences.size() == 0 || getLastSequenceMetadata().isCheckpointed()) { status = Status.PUBLISHING; } @@ -1021,37 +1003,6 @@ private void populateSequencesToPublish() } } - /** - * Start the continuous publishing executor for perpetual tasks - * Call this after the main reading loop starts - */ - private void startContinuousPublishing() - { - log.info("Starting continuous publishing for perpetual task [%s]", task.getId()); - - publishingTask = publishingExec.scheduleAtFixedRate( - this::performContinuousPublishing, - PUBLISH_INTERVAL_MS, - PUBLISH_INTERVAL_MS, - TimeUnit.MILLISECONDS - ); - } - - /** - * The method that runs periodically to check if there are sequences to be published - * for perpetual tasks - */ - private void performContinuousPublishing() - { - try { - maybePersistAndPublishSequences(committerSupplier); - waitForPublishAndHandoffCompletion(); - } - catch (Exception e) { - log.error(e, "Encountered exception while publishing in background thread"); - } - } - private TaskLockType getTaskLockType() { return TaskLocks.determineLockTypeForAppend(task.getContext()); @@ -1571,9 +1522,6 @@ public void stopForcefully() if (runThread != null) { runThread.interrupt(); } - if (publishingExec != null) { - publishingExec.shutdownNow(); - } } public void stopGracefully() @@ -1620,9 +1568,6 @@ public void stopGracefully() catch (Exception e) { throw new RuntimeException(e); } - if (publishingExec != null) { - publishingExec.shutdown(); - } } @POST From 54dbd68951c1676b0aac71810ade3c596a48c21a Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 17 Sep 2025 13:36:35 +0530 Subject: [PATCH 28/46] Fix the checkpoint <> task discovery race condition --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 3 +++ 1 file changed, 3 insertions(+) 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 096a0c2fd376..8aa42e469bf3 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 @@ -1117,7 +1117,10 @@ public void handle() throws ExecutionException, InterruptedException taskGroup.addNewCheckpoint(newCheckpoint); log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); } else { + // It might be possible that the task has not been discovered to the taskgroup yet and have received a checkpoint before hand, + // For now, I will attempt to repush the checkpoint request in the handler. log.warn("New checkpoint is null for taskGroup [%s]", taskGroupId); + addNotice(this); } } } From 733064be60afa5338bbdfc5a0b1f8b73be2aebee Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 17 Sep 2025 13:51:28 +0530 Subject: [PATCH 29/46] First embedded test success run --- .../indexing/KafkaClusterMetricsTest.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 29261e13d177..1da8625ee15c 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -182,11 +182,11 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() } @Test - @Timeout(600) - public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() + @Timeout(60) + public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() { final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 20; + final int expectedSegmentsHandedOff = 30; final int taskCount = 1; @@ -222,15 +222,13 @@ public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues supervisorId, cluster.callApi().postSupervisor(kafkaSupervisorSpec) ); - - overlord.latchableEmitter().waitForEvent( - event -> event.hasMetricName("task/autoScaler/scaleActionTime") - .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)) + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("task/autoScaler/scaleActionTime"), + agg -> agg.hasSumAtLeast(2) ); indexer.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("ingest/handoff/count") - .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)), + event -> event.hasMetricName("ingest/handoff/count"), agg -> agg.hasSumAtLeast(expectedSegmentsHandedOff) ); From 5907491929395ddc78907167fa3dab97d4a41286 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 17 Sep 2025 14:38:11 +0530 Subject: [PATCH 30/46] Make maven happy --- .../seekablestream/SeekableStreamIndexTask.java | 1 - .../SeekableStreamIndexTaskRunner.java | 1 + .../common/OrderedSequenceNumber.java | 3 ++- .../supervisor/SeekableStreamSupervisor.java | 14 ++++---------- .../metadata/MetadataStorageActionHandler.java | 4 ++-- .../SeekableStreamSupervisorStateTest.java | 2 ++ 6 files changed, 11 insertions(+), 14 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index f06177f5d2e2..683b43704749 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -39,7 +39,6 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.PendingSegmentAllocatingTask; -import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 2140403dd96a..56e0048db21f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -149,6 +149,7 @@ public abstract class SeekableStreamIndexTaskRunner newStartingSequences) + public TaskGroup withStartingSequences(Map newStartingSequences) { this.startingSequences = ImmutableMap.copyOf(newStartingSequences); return this; @@ -767,7 +767,7 @@ private boolean sendConfigUpdatesToTasks(Map> newP log.info("Sending configuration updates to the following partition groups %s", newPartitionGroups); List> updateFutures = new ArrayList<>(); Map latestCommittedOffsets = getOffsetsFromMetadataStorage(); - for (Map.Entry entry: activelyReadingTaskGroups.entrySet()) { + for (Map.Entry entry : activelyReadingTaskGroups.entrySet()) { int taskGroupId = entry.getKey(); TaskGroup existingTaskGroup = entry.getValue(); @@ -824,7 +824,7 @@ private ListenableFuture persistThenUpdateTaskConfig( } SeekableStreamIndexTask existingTask = (SeekableStreamIndexTask) existingTaskOpt.get(); - SeekableStreamIndexTask updatedTask = existingTask.withNewIoConfig(newIoConfig); + SeekableStreamIndexTask updatedTask = existingTask.withNewIoConfig(newIoConfig); log.info("Persisting updated config for task [%s] to storage", taskId); updateTaskIoConfigInQueueOrStorage(updatedTask); return updatedTask; @@ -1167,14 +1167,8 @@ private boolean isCheckpointSignatureValid( SeekableStreamDataSourceMetadata checkpointMetadata ) { - var checkpointPresentAlready = checkpoint; - var checkpointProposed = checkpointMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap(); - // TODO: Think about what we can do here for now. + // Earlier checkpoints may no longer have the same partitions. if (spec.usePerpetuallyRunningTasks()) { -// Set assignedPartitions = partitionGroups.getOrDefault( -// checkpointMetadata.getTaskGroupId(), -// Collections.emptySet() -// ); return true; } diff --git a/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 83259bee7d8e..1b871db80b69 100644 --- a/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -194,7 +194,7 @@ default List getTaskInfos( * Update the task entry with the given id. */ void update( - final String id, - final @NotNull Task entry + String id, + @NotNull Task entry ); } 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 d1ddedc46ed7..fd3eaee02e42 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 @@ -1111,6 +1111,7 @@ public void testCheckpointForActiveTaskGroup() throws InterruptedException, Json EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() { @Override public Duration getEmissionDuration() @@ -1323,6 +1324,7 @@ public void testEarlyStoppingOfTaskGroupBasedOnStopTaskCount() throws Interrupte EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() From 3852d8fc1154b067ee67ee6495ccb5443765a549 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 17 Sep 2025 20:27:50 +0530 Subject: [PATCH 31/46] Skip rollovers --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 4 ++++ 1 file changed, 4 insertions(+) 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 34b1dbedca35..d75cec4582d7 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 @@ -3657,6 +3657,10 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } else { stopTasksEarly = false; } + // If using perpetually running tasks, we should not stop tasks based on duration + if (spec.usePerpetuallyRunningTasks()) { + return; + } final AtomicInteger numStoppedTasks = new AtomicInteger(); // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing From e813699a0f905579be7b5b36ac8af8731338acda Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Wed, 17 Sep 2025 22:34:32 +0530 Subject: [PATCH 32/46] Fix failing embedded test --- .../testing/embedded/indexing/KafkaClusterMetricsTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 1da8625ee15c..62b759d8183e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -186,7 +186,7 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() { final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 30; + final int expectedSegmentsHandedOff = 300; final int taskCount = 1; @@ -487,7 +487,6 @@ private KafkaSupervisorSpec createKafkaSupervisor( .withConsumerProperties(kafkaServer.consumerProperties()) .withTaskCount(taskCount) .withAutoScalerConfig(autoScalerConfig) - .withTaskDuration(Period.hours(1)) ) .withId(supervisorId) .withUsePerpetuallyRunningTasks(usePerpetuallyRunningTasks) From 6874292d8576bca98ebdb240c8911cc07b3a81db Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 18 Sep 2025 13:36:46 +0530 Subject: [PATCH 33/46] WIP - improve coverage --- .../overlord/HeapMemoryTaskStorageTest.java | 19 +++ .../MetadataTaskStorageUpdateTaskTest.java | 117 ++++++++++++++++++ .../indexing/overlord/TaskQueueTest.java | 36 ++++++ 3 files changed, 172 insertions(+) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java index 7021d23fb50a..284538e2a875 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java @@ -94,4 +94,23 @@ public void testGetTaskInfos() Assert.assertEquals(1, taskInfosComplete.size()); Assert.assertEquals(task1.getId(), taskInfosComplete.get(0).getTask().getId()); } + + @Test + public void testUpdateTask() + { + final NoopTask originalTask = NoopTask.create(); + final TaskStatus runningStatus = TaskStatus.running(originalTask.getId()); + storage.insert(originalTask, runningStatus); + + final NoopTask updatedTask = new NoopTask(originalTask.getId(), null, "testDS", 5000, 0, null); + + storage.updateTask(updatedTask); + + TaskInfo taskInfo = storage.getTaskInfo(originalTask.getId()); + Assert.assertEquals(runningStatus, taskInfo.getStatus()); + Assert.assertEquals(5000L, ((NoopTask) taskInfo.getTask()).getRunTime()); + Assert.assertEquals("testDS", taskInfo.getTask().getDataSource()); + + Assert.assertThrows(IllegalStateException.class, () -> storage.updateTask(NoopTask.create())); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java new file mode 100644 index 000000000000..223e501dce01 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java @@ -0,0 +1,117 @@ +/* + * 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.overlord; + +import com.google.common.base.Optional; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.metadata.MetadataStorageActionHandler; +import org.apache.druid.metadata.MetadataStorageActionHandlerFactory; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; + +public class MetadataTaskStorageUpdateTaskTest +{ + private MetadataStorageConnector metadataStorageConnector; + private TaskStorageConfig taskStorageConfig; + private MetadataStorageActionHandlerFactory handlerFactory; + private MetadataStorageActionHandler handler; + private MetadataTaskStorage metadataTaskStorage; + + @Before + public void setUp() + { + metadataStorageConnector = EasyMock.createMock(MetadataStorageConnector.class); + taskStorageConfig = EasyMock.createMock(TaskStorageConfig.class); + handlerFactory = EasyMock.createMock(MetadataStorageActionHandlerFactory.class); + handler = EasyMock.createMock(MetadataStorageActionHandler.class); + + EasyMock.expect(handlerFactory.create()).andReturn(handler); + EasyMock.replay(handlerFactory); + + metadataTaskStorage = new MetadataTaskStorage( + metadataStorageConnector, + taskStorageConfig, + handlerFactory + ); + } + + @Test + public void testUpdateTaskSuccess() + { + final NoopTask task = NoopTask.create(); + + EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.of(task)); + handler.update(task.getId(), task); + EasyMock.expectLastCall(); + + EasyMock.replay(handler); + + metadataTaskStorage.updateTask(task); + + EasyMock.verify(handler); + } + + @Test(expected = ISE.class) + public void testUpdateTaskNotFound() + { + final NoopTask task = NoopTask.create(); + + EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.absent()); + + EasyMock.replay(handler); + + metadataTaskStorage.updateTask(task); + } + + @Test(expected = DruidException.class) + public void testUpdateTaskDruidException() + { + final NoopTask task = NoopTask.create(); + final DruidException druidException = DruidException.defensive("Test DruidException"); + + EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.of(task)); + handler.update(task.getId(), task); + EasyMock.expectLastCall().andThrow(druidException); + + EasyMock.replay(handler); + + metadataTaskStorage.updateTask(task); + } + + @Test(expected = RuntimeException.class) + public void testUpdateTaskGenericException() + { + final NoopTask task = NoopTask.create(); + final Exception genericException = new Exception("Test generic exception"); + + EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.of(task)); + handler.update(task.getId(), task); + EasyMock.expectLastCall().andThrow(genericException); + + EasyMock.replay(handler); + + metadataTaskStorage.updateTask(task); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index 3ee1fa065b28..c4e08d0054e8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -74,6 +74,7 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -690,6 +691,35 @@ public TaskStatus runTask(TaskToolbox toolbox) Assert.assertEquals(failedStatus, getTaskStorage().getStatus(task.getId()).get()); } + @Test + public void test_update_successfullyUpdates() + { + final TestTask originalTask = new TestTask("update-test-task", Intervals.of("2021-01-01/P1D")); + taskQueue.add(originalTask); + + final TestTask updatedTask = new TestTask("update-test-task", "updatedDatasource", Intervals.of("2021-01-01/P1D"), + ImmutableMap.of("testKey", "testValue")); + + taskQueue.update(updatedTask); + + final Optional updatedInfoOpt = taskQueue.getActiveTaskInfo(updatedTask.getId()); + Assert.assertTrue(updatedInfoOpt.isPresent()); + Assert.assertEquals("updatedDatasource", updatedInfoOpt.get().getTask().getDataSource()); + Assert.assertEquals("testValue", updatedInfoOpt.get().getTask().getContextValue("testKey")); + + final Optional taskInStorage = getTaskStorage().getTask(updatedTask.getId()); + Assert.assertTrue(taskInStorage.isPresent()); + Assert.assertEquals("updatedDatasource", taskInStorage.get().getDataSource()); + } + + @Test(expected = ISE.class) + public void test_update_nonExistingTask() + { + taskQueue.update(new TestTask("non-existing-task", Intervals.of("2021-01-01/P1D"))); + } + + + private HttpRemoteTaskRunner createHttpRemoteTaskRunner() { final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider @@ -744,6 +774,12 @@ private TestTask(String id, Interval interval, Map context) this.interval = interval; } + private TestTask(String id, String dataSource, Interval interval, Map context) + { + super(id, dataSource, context, IngestionMode.NONE); + this.interval = interval; + } + @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { From bad6b7f19210a12eba6d09c1c86e6d49097e80a5 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 18 Sep 2025 14:11:38 +0530 Subject: [PATCH 34/46] WIP - improve coverage 2 --- .../supervisor/SupervisorManager.java | 1 - ...eekableStreamIndexTaskClientAsyncImpl.java | 1 - ...bleStreamIndexTaskClientAsyncImplTest.java | 142 ++++++++++++ .../common/OrderedSequenceNumberTest.java | 211 ++++++++++++++++++ .../SQLMetadataStorageActionHandlerTest.java | 26 +++ 5 files changed, 379 insertions(+), 2 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 162902b036c7..168b956afd2d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -372,7 +372,6 @@ public boolean checkPointDataSourceMetadata( Preconditions.checkNotNull(supervisor, "supervisor could not be found"); final StreamSupervisor streamSupervisor = requireStreamSupervisor(supervisorId, "checkPoint"); - log.info("Checkpointing datasource metadata for supervisor [%s]", supervisorId); streamSupervisor.checkpoint(taskGroupId, previousDataSourceMetadata); return true; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index f1a742b6fb6a..60f8669e7eef 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -338,7 +338,6 @@ public ListenableFuture> pauseAndCheckp log.info("Task [%s] paused successfully & Checkpoint requested successffully", id); return deserializeOffsetsMap(r.getContent()); } else if (r.getStatus().equals(HttpResponseStatus.ACCEPTED)) { - // Return null, which triggers a loop later to wait for the task to enter PAUSED state. return null; } else { throw new ISE( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java index 72222ade56d6..c35440c4515a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java @@ -565,6 +565,148 @@ public void test_getParseErrorsAsync_empty() throws Exception Assert.assertNull(client.getParseErrorsAsync(TASK_ID).get()); } + @Test + public void test_pauseAndCheckpointAsync_immediateOk() throws Exception + { + final Map offsets = ImmutableMap.of(1, 3L); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(offsets) + ); + + Assert.assertEquals(offsets, client.pauseAndCheckpointAsync(TASK_ID).get()); + } + + @Test + public void test_pauseAndCheckpointAsync_immediateBadStatus() throws Exception + { + final Map offsets = ImmutableMap.of(1, 3L); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + HttpResponseStatus.CONTINUE, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(offsets) + ); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> client.pauseAndCheckpointAsync(TASK_ID).get() + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + MatcherAssert.assertThat( + e.getCause().getMessage(), + CoreMatchers.startsWith("Pause & Checkpoint request for task [the-task] failed with response [100 Continue]") + ); + } + + @Test + public void test_pauseAndCheckpointAsync_oneIteration() throws Exception + { + final Map offsets = ImmutableMap.of(1, 3L); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + HttpResponseStatus.ACCEPTED, + Collections.emptyMap(), + ByteArrays.EMPTY_ARRAY + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.PAUSED) + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(offsets) + ); + + Assert.assertEquals(offsets, client.pauseAndCheckpointAsync(TASK_ID).get()); + } + + @Test + public void test_pauseAndCheckpointAsync_notAvailable() throws Exception + { + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + new ServiceNotAvailableException(TASK_ID) + ); + + Assert.assertEquals(Collections.emptyMap(), client.pauseAndCheckpointAsync(TASK_ID).get()); + } + + @Test + public void test_updateConfigAsync_success() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/updateConfig") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + ByteArrays.EMPTY_ARRAY + ); + + Assert.assertEquals(true, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + + @Test + public void test_updateConfigAsync_httpError() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/updateConfig") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + new HttpResponseException( + new StringFullResponseHolder( + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.SERVICE_UNAVAILABLE), + StandardCharsets.UTF_8 + ) + ) + ); + + Assert.assertEquals(false, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + + @Test + public void test_updateConfigAsync_notAvailable() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/updateConfig") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + new ServiceNotAvailableException(TASK_ID) + ); + + Assert.assertEquals(false, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + + @Test + public void test_updateConfigAsync_closed() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/updateConfig") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + new ServiceClosedException(TASK_ID) + ); + + Assert.assertEquals(false, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + @Test public void test_serviceLocator_unknownTask() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java new file mode 100644 index 000000000000..47f80d17e18d --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java @@ -0,0 +1,211 @@ +/* + * 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.common; + +import org.junit.Assert; +import org.junit.Test; + +import javax.validation.constraints.NotNull; + +public class OrderedSequenceNumberTest +{ + @Test + public void test_isMoreToReadBeforeReadingRecord_exclusiveEnd_lessThan() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertTrue("Should have more to read when current < end with exclusive end", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void test_isMoreToReadBeforeReadingRecord_exclusiveEnd_equalTo() + { + TestSequenceNumber current = new TestSequenceNumber(10L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should NOT have more to read when current == end with exclusive end", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_exclusiveEnd_greaterThan() + { + TestSequenceNumber current = new TestSequenceNumber(15L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should NOT have more to read when current > end with exclusive end", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_inclusiveEnd_lessThan() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertTrue("Should have more to read when current < end with inclusive end", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_inclusiveEnd_equalTo() + { + TestSequenceNumber current = new TestSequenceNumber(10L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertTrue("Should have more to read when current == end with inclusive end", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_inclusiveEnd_greaterThan() + { + TestSequenceNumber current = new TestSequenceNumber(15L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should NOT have more to read when current > end with inclusive end", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullEndSequenceNumber_exclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(null, false); + + Assert.assertFalse("Should return false when end sequence number is null", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullEndSequenceNumber_inclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(null, false); + + Assert.assertFalse("Should return false when end sequence number is null", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullCurrentSequenceNumber_exclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(null, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should return false when current sequence number is null", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullCurrentSequenceNumber_inclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(null, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should return false when current sequence number is null", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_bothNull() + { + TestSequenceNumber current = new TestSequenceNumber(null, false); + TestSequenceNumber end = new TestSequenceNumber(null, false); + + Assert.assertFalse("Should return false when both sequence numbers are null", + current.isMoreToReadBeforeReadingRecord(end, true)); + Assert.assertFalse("Should return false when both sequence numbers are null", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_exceptionHandling() + { + TestExceptionSequenceNumber current = new TestExceptionSequenceNumber(5L, false); + TestExceptionSequenceNumber end = new TestExceptionSequenceNumber(10L, false); + + Assert.assertFalse("Should return false when comparison throws exception", + current.isMoreToReadBeforeReadingRecord(end, true)); + Assert.assertFalse("Should return false when comparison throws exception", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_differentExclusivityFlags() + { + TestSequenceNumber currentExclusive = new TestSequenceNumber(5L, true); + TestSequenceNumber currentInclusive = new TestSequenceNumber(5L, false); + TestSequenceNumber endExclusive = new TestSequenceNumber(10L, true); + TestSequenceNumber endInclusive = new TestSequenceNumber(10L, false); + + // Test different combinations with exclusive end + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endExclusive, true)); + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endInclusive, true)); + Assert.assertTrue("Should work with different exclusivity flags", + currentInclusive.isMoreToReadBeforeReadingRecord(endExclusive, true)); + + // Test different combinations with inclusive end + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endExclusive, false)); + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endInclusive, false)); + Assert.assertTrue("Should work with different exclusivity flags", + currentInclusive.isMoreToReadBeforeReadingRecord(endExclusive, false)); + } + + /** + * Test implementation of OrderedSequenceNumber for Long values + */ + private static class TestSequenceNumber extends OrderedSequenceNumber + { + public TestSequenceNumber(Long sequenceNumber, boolean isExclusive) + { + super(sequenceNumber, isExclusive); + } + + @Override + public int compareTo(@NotNull OrderedSequenceNumber o) + { + return this.get().compareTo(o.get()); + } + } + + /** + * Test implementation that throws exceptions on comparison + */ + private static class TestExceptionSequenceNumber extends OrderedSequenceNumber + { + public TestExceptionSequenceNumber(Long sequenceNumber, boolean isExclusive) + { + super(sequenceNumber, isExclusive); + } + + @Override + public int compareTo(@NotNull OrderedSequenceNumber o) + { + throw new RuntimeException("Comparison not supported"); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 604f682fb66a..3aee64437fce 100644 --- a/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -556,4 +556,30 @@ private void verifyTaskStatus(TaskStatus expected, TaskStatus actual) { Assert.assertEquals(expected, actual); } + + @Test + public void testUpdateTask() + { + Task originalTask = NoopTask.create(); + TaskStatus status = TaskStatus.running(originalTask.getId()); + final String entryId = originalTask.getId(); + + handler.insert(entryId, DateTimes.of("2014-01-01T00:00:00.123"), "testDataSource", originalTask, true, status, "type", "group"); + + Assert.assertEquals(Optional.of(originalTask), handler.getEntry(entryId)); + Assert.assertEquals("none", handler.getEntry(entryId).get().getDataSource()); + + Task updatedTask = new NoopTask(entryId, "updatedGroup", "updatedDataSource", 5000L, 0L, + ImmutableMap.of("testKey", "testValue")); + + handler.update(entryId, updatedTask); + + Optional retrievedTask = handler.getEntry(entryId); + Assert.assertTrue(retrievedTask.isPresent()); + Assert.assertEquals(entryId, retrievedTask.get().getId()); + Assert.assertEquals("updatedDataSource", retrievedTask.get().getDataSource()); + Assert.assertEquals("updatedGroup", retrievedTask.get().getGroupId()); + Assert.assertEquals("testValue", retrievedTask.get().getContextValue("testKey")); + Assert.assertEquals(5000L, ((NoopTask) retrievedTask.get()).getRunTime()); + } } From 1dab56443f8fe645f00832a58c4bb6986863cf1c Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 18 Sep 2025 17:55:06 +0530 Subject: [PATCH 35/46] Fix tests and implcitly turn on release locks on handoff to true --- .../indexing/KafkaClusterMetricsTest.java | 2 +- .../RabbitStreamSupervisorTuningConfig.java | 3 +- .../KafkaSupervisorTuningConfig.java | 4 +- .../kafka/KafkaIndexTaskTuningConfigTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 14 +++---- .../KinesisSupervisorTuningConfig.java | 2 +- .../KinesisIndexTaskTuningConfigTest.java | 2 +- .../supervisor/KinesisSupervisorTest.java | 10 ++--- .../supervisor/SeekableStreamSupervisor.java | 8 ++-- .../SeekableStreamSupervisorTuningConfig.java | 2 +- .../SeekableStreamSupervisorSpecTest.java | 7 +++- .../SeekableStreamSupervisorStateTest.java | 42 ++++++++++++++----- 12 files changed, 63 insertions(+), 35 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 62b759d8183e..76d2db5ad3f7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -183,7 +183,7 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() @Test @Timeout(60) - public void test_ingest30kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() + public void test_ingest300kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() { final int maxRowsPerSegment = 1000; final int expectedSegmentsHandedOff = 300; diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java index a2667026fffd..70b886df9ad2 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.incremental.AppendableIndexSpec; @@ -219,7 +220,7 @@ public String toString() } @Override - public RabbitStreamIndexTaskTuningConfig convertToTaskTuningConfig() + public RabbitStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new RabbitStreamIndexTaskTuningConfig( getAppendableIndexSpec(), diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index c4a21674d301..1a177128e6d4 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -213,7 +213,7 @@ public String toString() } @Override - public KafkaIndexTaskTuningConfig convertToTaskTuningConfig() + public KafkaIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new KafkaIndexTaskTuningConfig( getAppendableIndexSpec(), @@ -237,7 +237,7 @@ public KafkaIndexTaskTuningConfig convertToTaskTuningConfig() getMaxSavedParseExceptions(), getNumPersistThreads(), getMaxColumnsToMerge(), - isReleaseLocksOnHandoff() + usePerpetuallyRunningTasks ); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 54ae10080a7b..abab48c7ab10 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -144,7 +144,7 @@ public void testConvert() .withReportParseExceptions(true) .withMaxColumnsToMerge(5) .build(); - KafkaIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(); + KafkaIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(null); Assert.assertEquals(original.getAppendableIndexSpec(), copy.getAppendableIndexSpec()); Assert.assertEquals(1, copy.getMaxRowsInMemory()); 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 2ed14296f273..9ab6a0d5791b 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 @@ -397,7 +397,7 @@ public SeekableStreamIndexTaskClient build( KafkaIndexTask task = captured.getValue(); Assert.assertEquals(KafkaSupervisorTest.dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -536,7 +536,7 @@ public void testNoInitialState() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -1904,7 +1904,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception for (Task task : captured.getValues()) { KafkaIndexTask kafkaIndexTask = (KafkaIndexTask) task; Assert.assertEquals(dataSchema, kafkaIndexTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), kafkaIndexTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), kafkaIndexTask.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = kafkaIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -2014,7 +2014,7 @@ public void testDiscoverExistingPublishingTask() throws Exception KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -2148,7 +2148,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -4296,7 +4296,7 @@ public void testFailedInitializationAndRecovery() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -5614,7 +5614,7 @@ private KafkaIndexTask createKafkaIndexTask( minimumMessageTime, maximumMessageTime, schema, - tuningConfig.convertToTaskTuningConfig() + tuningConfig.convertToTaskTuningConfig(null) ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 1a11f8d658b7..0a023caf9ed7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -252,7 +252,7 @@ public String toString() } @Override - public KinesisIndexTaskTuningConfig convertToTaskTuningConfig() + public KinesisIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new KinesisIndexTaskTuningConfig( getAppendableIndexSpec(), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 375e26e2ed28..82e1a117aa9e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -331,7 +331,7 @@ public void testConvert() null, null ); - KinesisIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(); + KinesisIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(null); Assert.assertEquals(original.getAppendableIndexSpec(), copy.getAppendableIndexSpec()); Assert.assertEquals(1, copy.getMaxRowsInMemory()); 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 d24c309e7c89..4a83cf81e393 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 @@ -263,7 +263,7 @@ public void testNoInitialState() throws Exception KinesisIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KinesisIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -1593,7 +1593,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception for (Task task : secondTasks.getValues()) { KinesisIndexTask kinesisIndexTask = (KinesisIndexTask) task; Assert.assertEquals(dataSchema, kinesisIndexTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), kinesisIndexTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), kinesisIndexTask.getTuningConfig()); KinesisIndexTaskIOConfig taskConfig = kinesisIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -1746,7 +1746,7 @@ public void testDiscoverExistingPublishingTask() throws Exception KinesisIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); @@ -1896,7 +1896,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KinesisIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); @@ -5603,7 +5603,7 @@ private KinesisIndexTask createKinesisIndexTask( minimumMessageTime, maximumMessageTime, dataSchema, - (KinesisIndexTaskTuningConfig) tuningConfig.convertToTaskTuningConfig() + (KinesisIndexTaskTuningConfig) tuningConfig.convertToTaskTuningConfig(null) ); } 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 20df7379054f..d362385da9f4 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 @@ -1299,7 +1299,7 @@ public SeekableStreamSupervisor( this.ioConfig = spec.getIoConfig(); this.autoScalerConfig = ioConfig.getAutoScalerConfig(); this.tuningConfig = spec.getTuningConfig(); - this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig(); + this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()); this.supervisorId = spec.getId(); this.exec = Execs.singleThreaded(StringUtils.encodeForFormat(supervisorTag)); this.scheduledExec = Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(supervisorTag) + "-Scheduler-%d"); @@ -1838,7 +1838,7 @@ public Map> getStats() public List getParseErrors() { try { - if (spec.getSpec().getTuningConfig().convertToTaskTuningConfig().getMaxParseExceptions() <= 0) { + if (spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()).getMaxParseExceptions() <= 0) { return ImmutableList.of(); } lastKnownParseErrors = getCurrentParseErrors(); @@ -2003,11 +2003,11 @@ private List getCurrentParseErrors() } } - SeekableStreamIndexTaskTuningConfig ss = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig ss = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()); SeekableStreamSupervisorIOConfig oo = spec.getSpec().getIOConfig(); // store a limited number of parse exceptions, keeping the most recent ones - int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig().getMaxSavedParseExceptions() * + int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()).getMaxSavedParseExceptions() * spec.getSpec().getIOConfig().getTaskCount(); parseErrorLimit = Math.min(parseErrorLimit, parseErrorsTreeSet.size()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index d1f10e77efa8..7f7e66506d77 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -55,5 +55,5 @@ static Duration defaultDuration(final Period period, final String theDefault) @JsonProperty Duration getOffsetFetchPeriod(); - SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index 1ccc1458f99d..d52ae0c9056b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -509,7 +509,7 @@ public Duration getOffsetFetchPeriod() } @Override - public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new SeekableStreamIndexTaskTuningConfig( null, @@ -817,6 +817,7 @@ public void testSeekableStreamSupervisorSpecWithScaleOutAlreadyAtMax() throws In EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -876,6 +877,7 @@ public void test_dynamicAllocationNotice_skipsScalingAndEmitsReason_ifTasksArePu EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -942,6 +944,7 @@ public void testSeekableStreamSupervisorSpecWithNoScalingOnIdleSupervisor() thro EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1147,6 +1150,7 @@ public void testSeekableStreamSupervisorSpecWithScaleInAlreadyAtMin() throws Int EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1227,6 +1231,7 @@ public void testSeekableStreamSupervisorSpecWithScaleDisable() throws Interrupte EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(this.seekableStreamSupervisorIOConfig).anyTimes(); 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 fd3eaee02e42..27fd7e8aa307 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 @@ -207,6 +207,7 @@ public void setupTest() public void testRunning() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -249,6 +250,7 @@ public void testRunningStreamGetSequenceNumberReturnsNull() EasyMock.expect(recordSupplier.getAssignment()).andReturn(ImmutableSet.of(SHARD0_PARTITION)).anyTimes(); EasyMock.expect(recordSupplier.getLatestSequenceNumber(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -298,6 +300,7 @@ public void testRunningStreamGetSequenceNumberReturnsNull() public void testAddDiscoveredTaskToPendingCompletionTaskGroups() throws Exception { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -402,6 +405,7 @@ public void testAddDiscoveredTaskToPendingCompletionTaskGroups() throws Exceptio public void testAddDiscoveredTaskToPendingCompletionMultipleTaskGroups() throws Exception { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -473,6 +477,7 @@ public void testAddDiscoveredTaskToPendingCompletionMultipleTaskGroups() throws public void testConnectingToStreamFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)) .andThrow(new StreamException(new IllegalStateException(EXCEPTION_MSG))) .anyTimes(); @@ -528,6 +533,7 @@ public void testConnectingToStreamFail() public void testConnectingToStreamFailRecoveryFailRecovery() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)) .andThrow(new StreamException(new IllegalStateException())) .times(3); @@ -602,6 +608,7 @@ public void testConnectingToStreamFailRecoveryFailRecovery() public void testDiscoveringInitialTasksFailRecoveryFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).times(6); @@ -711,6 +718,7 @@ public Duration getEmissionDuration() } }).anyTimes(); EasyMock.expect(spec.getType()).andReturn("test").anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); @@ -784,6 +792,7 @@ public void testIdleOnStartUpAndTurnsToRunningAfterLagUpdates() EasyMock.reset(spec); EasyMock.expect(spec.getId()).andReturn(SUPERVISOR_ID).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getContextValue("tags")).andReturn("").anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(new SeekableStreamSupervisorIOConfig( @@ -859,6 +868,7 @@ public Duration getEmissionDuration() public void testCreatingTasksFailRecoveryFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); @@ -937,6 +947,7 @@ public void testCreatingTasksFailRecoveryFail() public void testSuspended() { EasyMock.expect(spec.isSuspended()).andReturn(true).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -976,6 +987,7 @@ public void testSuspended() public void testStopping() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -1020,6 +1032,7 @@ public void testStopGracefully() throws Exception EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); taskRunner.unregisterListener("testSupervisorId"); indexTaskClient.close(); @@ -1039,6 +1052,7 @@ public void testStopGracefully() throws Exception public void testStoppingGracefully() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -1136,7 +1150,7 @@ public Duration getEmissionDuration() ioConfig ); - SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(false); TreeMap> sequenceOffsets = new TreeMap<>(); sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L)); @@ -1341,7 +1355,7 @@ public Duration getEmissionDuration() EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); - SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(false); TreeMap> sequenceOffsets = new TreeMap<>(); sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L)); @@ -1556,6 +1570,7 @@ public void testSupervisorStopTaskGroupEarly() throws JsonProcessingException, I EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andStubReturn(false); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() { @Override @@ -1572,7 +1587,7 @@ public Duration getEmissionDuration() EasyMock.expect(streamingTaskRunner.getStatus()).andReturn(null); EasyMock.expect(streamingTaskRunner.getStatus()).andReturn(SeekableStreamIndexTaskRunner.Status.NOT_STARTED); - SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(false); TreeMap> sequenceOffsets = new TreeMap<>(); sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L)); @@ -1730,7 +1745,6 @@ public void testEmitRecordLag() throws Exception public void testEmitTimeLag() throws Exception { expectEmitterSupervisor(false); - CountDownLatch latch = new CountDownLatch(1); TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( latch, @@ -1763,7 +1777,6 @@ public void testEmitTimeLag() throws Exception public void testEmitNoticesQueueSize() throws Exception { expectEmitterSupervisor(false); - CountDownLatch latch = new CountDownLatch(1); TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( latch, @@ -1827,7 +1840,6 @@ public void testEmitNoticesTime() throws Exception public void testEmitNoLagWhenSuspended() throws Exception { expectEmitterSupervisor(true); - CountDownLatch latch = new CountDownLatch(1); TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( latch, @@ -1862,6 +1874,7 @@ public void testEmitNoLagWhenSuspended() throws Exception public void testGetStats() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexTaskClient.getMovingAveragesAsync("task1")) .andReturn(Futures.immediateFuture(ImmutableMap.of("prop1", "val1"))) .times(1); @@ -1909,6 +1922,7 @@ public void testGetStats() public void testSupervisorResetAllWithCheckpoints() throws InterruptedException { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(SUPERVISOR_ID)).andReturn( true ); @@ -1962,6 +1976,7 @@ public void testSupervisorResetOneTaskSpecificOffsetsWithCheckpoints() throws In ) ) ); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata(SUPERVISOR_ID, new TestSeekableStreamDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>( STREAM, @@ -2008,7 +2023,7 @@ public void testSupervisorResetOneTaskSpecificOffsetsWithCheckpoints() throws In public void testRegisterNewVersionOfPendingSegment() { EasyMock.expect(spec.isSuspended()).andReturn(false); - + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); Capture captured0 = Capture.newInstance(CaptureType.FIRST); Capture captured1 = Capture.newInstance(CaptureType.FIRST); EasyMock.expect( @@ -2090,7 +2105,7 @@ public void testSupervisorResetSpecificOffsetsTasksWithCheckpoints() throws Inte final ImmutableMap checkpointOffsets = ImmutableMap.of("0", "5", "1", "6", "2", "100"); final ImmutableMap resetOffsets = ImmutableMap.of("0", "10", "1", "8"); final ImmutableMap expectedOffsets = ImmutableMap.of("0", "10", "1", "8", "2", "100"); - + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( @@ -2170,6 +2185,7 @@ public void testSupervisorResetOffsetsWithNoCheckpoints() throws InterruptedExce final ImmutableMap expectedOffsets = ImmutableMap.copyOf(resetOffsets); EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn(null); EasyMock.expect(indexerMetadataStorageCoordinator.insertDataSourceMetadata(SUPERVISOR_ID, new TestSeekableStreamDataSourceMetadata( @@ -2243,6 +2259,7 @@ public void testSupervisorResetWithNoPartitions() throws IOException, Interrupte final ImmutableMap expectedOffsets = ImmutableMap.of("0", "5", "1", "6"); EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( new TestSeekableStreamDataSourceMetadata( @@ -2308,6 +2325,7 @@ public void testSupervisorResetWithNewPartition() throws IOException, Interrupte final ImmutableMap expectedOffsets = ImmutableMap.of("0", "5", "1", "6", "2", "20"); EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( new TestSeekableStreamDataSourceMetadata( @@ -2371,6 +2389,7 @@ public void testSupervisorResetWithNewPartition() throws IOException, Interrupte public void testSupervisorNoResetDataSourceMetadata() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2410,6 +2429,7 @@ public void testSupervisorNoResetDataSourceMetadata() public void testSupervisorResetWithInvalidStartSequenceMetadata() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2460,6 +2480,7 @@ public void testSupervisorResetWithInvalidStartSequenceMetadata() public void testSupervisorResetInvalidStream() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2506,7 +2527,6 @@ public void testSupervisorResetInvalidStream() public void testStaleOffsetsNegativeLagNotEmitted() throws Exception { expectEmitterSupervisor(false); - CountDownLatch latch = new CountDownLatch(1); final TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( @@ -2549,6 +2569,7 @@ private void validateSupervisorStateAfterResetOffsets( public void testScheduleReporting() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); DruidMonitorSchedulerConfig config = new DruidMonitorSchedulerConfig(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(config).times(2); ScheduledExecutorService executorService = EasyMock.createMock(ScheduledExecutorService.class); @@ -2651,6 +2672,7 @@ private void expectEmitterSupervisor(boolean suspended) spec = createMock(SeekableStreamSupervisorSpec.class); EasyMock.expect(spec.getId()).andReturn(SUPERVISOR_ID).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(new SeekableStreamSupervisorIOConfig( "stream", @@ -2865,7 +2887,7 @@ public Duration getOffsetFetchPeriod() } @Override - public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new SeekableStreamIndexTaskTuningConfig( null, From 2054c06f6a6a62d088021553206d17e70ced2fae Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 18 Sep 2025 22:30:00 +0530 Subject: [PATCH 36/46] Fix scale in issues --- .../indexing/KafkaClusterMetricsTest.java | 72 ++++++++++++++++++- .../RabbitStreamSupervisorTuningConfig.java | 1 - .../SeekableStreamIndexTaskRunner.java | 15 +++- .../supervisor/SeekableStreamSupervisor.java | 12 ++-- 4 files changed, 88 insertions(+), 12 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 76d2db5ad3f7..510185a965f9 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -183,10 +183,10 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() @Test @Timeout(60) - public void test_ingest300kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() + public void test_ingest50kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() { final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 300; + final int expectedSegmentsHandedOff = 50; final int taskCount = 1; @@ -366,6 +366,74 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); } + @Test + @Timeout(60) + public void test_ingest50kRows_ofSelfClusterMetricsWithScaleIns_andVerifyValues() + { + final int maxRowsPerSegment = 1000; + final int expectedSegmentsHandedOff = 50; + + final int initialTaskCount = 3; + + // Submit and start a supervisor with scale-in configuration + final String supervisorId = dataSource + "_supe"; + AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() + .withLagCollectionIntervalMillis(500) + .withLagCollectionRangeMillis(1000) + .withEnableTaskAutoScaler(true) + .withScaleActionPeriodMillis(10000) + .withScaleActionStartDelayMillis(5000) + .withScaleOutThreshold(10000) + .withScaleInThreshold(1) + .withTriggerScaleOutFractionThreshold(0.9) + .withTriggerScaleInFractionThreshold(0.001) + .withTaskCountMax(initialTaskCount) + .withTaskCountStart(initialTaskCount) + .withScaleOutStep(0) + .withScaleInStep(1) + .withMinTriggerScaleActionFrequencyMillis(10000) + .withStopTaskCountRatio(1.0) + .build(); + + final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( + supervisorId, + initialTaskCount, + maxRowsPerSegment, + autoScalerConfig, + true + ); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("task/autoScaler/scaleActionTime"), + agg -> agg.hasSumAtLeast(2) + ); + + indexer.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("ingest/handoff/count"), + agg -> agg.hasSumAtLeast(expectedSegmentsHandedOff) + ); + + final int numSegments = Integer.parseInt( + cluster.runSql("SELECT COUNT(*) FROM sys.segments WHERE datasource = '%s'", dataSource) + ); + Assertions.assertTrue(numSegments >= expectedSegmentsHandedOff); + + final int numRows = Integer.parseInt( + cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) + ); + Assertions.assertTrue(numRows >= expectedSegmentsHandedOff * maxRowsPerSegment); + + verifyIngestedMetricCountMatchesEmittedCount("jvm/pool/committed", coordinator); + verifyIngestedMetricCountMatchesEmittedCount("coordinator/time", coordinator); + + cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); + } + @Test @Timeout(120) public void test_ingestClusterMetrics_compactionSkipsLockedIntervals() diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java index 70b886df9ad2..ea2bc7b25e1a 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskTuningConfig; -import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.incremental.AppendableIndexSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 56e0048db21f..056c9a3fe3e9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -1769,8 +1769,15 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt createNewSequenceFromIoConfig(newIoConfig); assignment = assignPartitions(recordSupplier); - possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); - seekToStartingSequence(recordSupplier, assignment); + boolean shouldResume = true; + if (!assignment.isEmpty()) { + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); + seekToStartingSequence(recordSupplier, assignment); + } else { + // if there is no assignment, It means that there was no partition assigned to this task after scaling down. + pause(); + shouldResume = false; + } log.info("Config updated to [%s]", this.ioConfig); toolbox.getEmitter().emit(ServiceMetricEvent.builder() @@ -1779,7 +1786,9 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) .setMetric("task/config/update/success", 1) .build(ImmutableMap.of())); - resume(); + if (shouldResume) { + resume(); + } waitForConfigUpdate.set(false); return Response.ok().build(); } 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 d362385da9f4..7d8ce789384e 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 @@ -778,7 +778,7 @@ private boolean sendConfigUpdatesToTasks(Map> newP int taskGroupId = entry.getKey(); TaskGroup existingTaskGroup = entry.getValue(); - Set partitionsForThisTask = new HashSet<>(newPartitionGroups.get(taskGroupId)); + Set partitionsForThisTask = new HashSet<>(newPartitionGroups.getOrDefault(taskGroupId, Set.of())); SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig( partitionsForThisTask, existingTaskGroup, @@ -876,16 +876,16 @@ private void handleObsoleteTaskGroups(Set newTaskGroupIds) for (Integer obsoleteTaskGroupId : obsoleteTaskGroupIds) { TaskGroup obsoleteTaskGroup = activelyReadingTaskGroups.get(obsoleteTaskGroupId); if (obsoleteTaskGroup != null) { - log.info("Pausing tasks in obsolete task group [%d]: %s", obsoleteTaskGroupId, obsoleteTaskGroup.taskIds()); + log.info("Gracefully shutting down tasks in obsolete task group [%d]: %s", obsoleteTaskGroupId, obsoleteTaskGroup.taskIds()); - // Pause all tasks in the obsolete task group + // Gracefully shut down all tasks in the obsolete task group for (String taskId : obsoleteTaskGroup.taskIds()) { try { - taskClient.pauseAsync(taskId); - log.info("Requested pause for task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); + killTaskWithSuccess(taskId, "Gracefully shutting down task in obsolete task group [%d] during scale down", obsoleteTaskGroupId); + log.info("Requested graceful shutdown for task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); } catch (Exception e) { - log.error(e, "Failed to pause task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); + log.error(e, "Failed to gracefully shut down task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); } } From 290bc6bdc28acd9d70365f309f55911177f715e5 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 19 Sep 2025 01:11:13 +0530 Subject: [PATCH 37/46] Scale down the embedded tests --- .../indexing/KafkaClusterMetricsTest.java | 8 +-- .../indexing/kafka/KafkaIndexTaskTest.java | 54 +++++++++++++++++++ .../TaskConfigUpdateRequestTest.java | 53 ++++++++++++++++++ 3 files changed, 111 insertions(+), 4 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 510185a965f9..7261e0168012 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -183,10 +183,10 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() @Test @Timeout(60) - public void test_ingest50kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() + public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() { final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 50; + final int expectedSegmentsHandedOff = 20; final int taskCount = 1; @@ -368,10 +368,10 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip @Test @Timeout(60) - public void test_ingest50kRows_ofSelfClusterMetricsWithScaleIns_andVerifyValues() + public void test_ingest20kRows_ofSelfClusterMetricsWithScaleIns_andVerifyValues() { final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 50; + final int expectedSegmentsHandedOff = 20; final int initialTaskCount = 3; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 826963a20ec3..81a06a453bd0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -3361,6 +3361,60 @@ public void testCompletionReportMultiplePartitionStats() throws Exception Assert.assertTrue(reportData.getRecordsProcessed().values().containsAll(ImmutableSet.of(6L, 2L))); } + @Test + public void testWithNewIoConfig() throws Exception + { + final KafkaIndexTaskIOConfig originalIoConfig = new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null, + INPUT_FORMAT, + null, + Duration.standardHours(2).getStandardMinutes() + ); + + final KafkaIndexTask originalTask = createTask("testTask", originalIoConfig); + + final KafkaIndexTaskIOConfig newIoConfig = new KafkaIndexTaskIOConfig( + 1, + "sequence1", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + false, + null, + null, + INPUT_FORMAT, + null, + Duration.standardHours(1).getStandardMinutes() + ); + + final KafkaIndexTask newTask = (KafkaIndexTask) originalTask.withNewIoConfig(newIoConfig); + + Assert.assertNotSame("New task should be a different instance", originalTask, newTask); + Assert.assertEquals("Task ID should be preserved", originalTask.getId(), newTask.getId()); + Assert.assertEquals("Supervisor ID should be preserved", originalTask.getSupervisorId(), newTask.getSupervisorId()); + Assert.assertEquals("Task resource should be preserved", originalTask.getTaskResource(), newTask.getTaskResource()); + Assert.assertEquals("Data schema should be preserved", originalTask.getDataSchema(), newTask.getDataSchema()); + Assert.assertEquals("Tuning config should be preserved", originalTask.getTuningConfig(), newTask.getTuningConfig()); + Assert.assertEquals("Context should be preserved", originalTask.getContext(), newTask.getContext()); + Assert.assertEquals("Perpetually running flag should be preserved", originalTask.isPerpetuallyRunning(), newTask.isPerpetuallyRunning()); + + final KafkaIndexTaskIOConfig actualNewIoConfig = newTask.getIOConfig(); + Assert.assertEquals("IO config should be updated", newIoConfig, actualNewIoConfig); + Assert.assertNotEquals("IO config should be different from original", originalIoConfig, actualNewIoConfig); + + Assert.assertEquals("Start sequence should be updated", 5L, (long) actualNewIoConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(new KafkaTopicPartition(false, topic, 0))); + Assert.assertEquals("End sequence should be updated", 10L, (long) actualNewIoConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(new KafkaTopicPartition(false, topic, 0))); + } + public static class TestKafkaInputFormat implements InputFormat { final InputFormat baseInputFormat; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java new file mode 100644 index 000000000000..b05a48f28ff1 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java @@ -0,0 +1,53 @@ +/* + * 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; + +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +public class TaskConfigUpdateRequestTest +{ + @Test + public void testTaskConfigUpdateRequest() + { + SeekableStreamIndexTaskIOConfig mockIoConfig = EasyMock.createMock(SeekableStreamIndexTaskIOConfig.class); + EasyMock.replay(mockIoConfig); + + TaskConfigUpdateRequest request = new TaskConfigUpdateRequest(mockIoConfig); + Assert.assertEquals(mockIoConfig, request.getIoConfig()); + + TaskConfigUpdateRequest nullRequest = new TaskConfigUpdateRequest(null); + Assert.assertNull(nullRequest.getIoConfig()); + + TaskConfigUpdateRequest request2 = new TaskConfigUpdateRequest(mockIoConfig); + Assert.assertEquals(request, request2); + Assert.assertEquals(request.hashCode(), request2.hashCode()); + + Assert.assertNotEquals(request, nullRequest); + Assert.assertNotEquals(request.hashCode(), nullRequest.hashCode()); + + String toString = request.toString(); + Assert.assertTrue(toString.contains("TaskConfigUpdateRequest")); + Assert.assertTrue(toString.contains("ioConfig")); + + EasyMock.verify(mockIoConfig); + } +} \ No newline at end of file From cacabbf5cf3519ddeb15f634859674d145dc2d14 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 19 Sep 2025 10:32:14 +0530 Subject: [PATCH 38/46] See if removing router fixes js error --- .../testing/embedded/indexing/KafkaClusterMetricsTest.java | 3 +-- .../indexing/seekablestream/TaskConfigUpdateRequestTest.java | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 7261e0168012..a1efd5cd9d76 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -127,8 +127,7 @@ public void stop() .addServer(overlord) .addServer(indexer) .addServer(broker) - .addServer(historical) - .addServer(new EmbeddedRouter()); + .addServer(historical); return cluster; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java index b05a48f28ff1..b853fb03aadb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java @@ -50,4 +50,4 @@ public void testTaskConfigUpdateRequest() EasyMock.verify(mockIoConfig); } -} \ No newline at end of file +} From 819973dc41b6b1ecdca639372ce44153b6636105 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 19 Sep 2025 11:15:57 +0530 Subject: [PATCH 39/46] Attempt clearing console from the deps --- embedded-tests/pom.xml | 8 -------- .../embedded/indexing/KafkaClusterMetricsTest.java | 1 - 2 files changed, 9 deletions(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index fb83b4c7421b..36cd80a8fcd0 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -228,14 +228,6 @@ ${project.parent.version} test - - - org.apache.druid - web-console - ${project.parent.version} - test - - org.apache.datasketches diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index a1efd5cd9d76..d8bcb4c34108 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -45,7 +45,6 @@ import org.apache.druid.testing.embedded.EmbeddedHistorical; import org.apache.druid.testing.embedded.EmbeddedIndexer; import org.apache.druid.testing.embedded.EmbeddedOverlord; -import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.emitter.LatchableEmitterModule; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.joda.time.Period; From 6c11f9b0c3ce4e0d421a30890a5c8cbe0a008bfe Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 19 Sep 2025 16:39:09 +0530 Subject: [PATCH 40/46] Should fix flaky embedded test --- embedded-tests/pom.xml | 8 ++++++++ .../embedded/indexing/KafkaClusterMetricsTest.java | 10 +++++++--- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 36cd80a8fcd0..fb83b4c7421b 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -228,6 +228,14 @@ ${project.parent.version} test + + + org.apache.druid + web-console + ${project.parent.version} + test + + org.apache.datasketches diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index d8bcb4c34108..fdf144dc58d7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -45,6 +45,7 @@ import org.apache.druid.testing.embedded.EmbeddedHistorical; import org.apache.druid.testing.embedded.EmbeddedIndexer; import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.emitter.LatchableEmitterModule; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.joda.time.Period; @@ -126,7 +127,8 @@ public void stop() .addServer(overlord) .addServer(indexer) .addServer(broker) - .addServer(historical); + .addServer(historical) + .addServer(new EmbeddedRouter()); return cluster; } @@ -226,7 +228,8 @@ public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues ); indexer.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("ingest/handoff/count"), + event -> event.hasMetricName("ingest/handoff/count") + .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)), agg -> agg.hasSumAtLeast(expectedSegmentsHandedOff) ); @@ -412,7 +415,8 @@ public void test_ingest20kRows_ofSelfClusterMetricsWithScaleIns_andVerifyValues( ); indexer.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("ingest/handoff/count"), + event -> event.hasMetricName("ingest/handoff/count") + .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)), agg -> agg.hasSumAtLeast(expectedSegmentsHandedOff) ); From 5d098062bf602a6c5de98d531351c3c379c2a426 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Mon, 22 Sep 2025 09:05:19 +0530 Subject: [PATCH 41/46] Remove complaining ioConfig and add kafka supervisor test --- .../supervisor/RabbitStreamSupervisor.java | 10 +- .../kafka/supervisor/KafkaSupervisor.java | 28 +-- .../kafka/supervisor/KafkaSupervisorTest.java | 222 +++++++++++++++++- .../supervisor/SeekableStreamSupervisor.java | 3 - 4 files changed, 239 insertions(+), 24 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 b4a5a14b8440..60446d7aa7a7 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 @@ -121,9 +121,9 @@ protected RecordSupplier setupRecordSupplier() RabbitStreamIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig(); return new RabbitStreamRecordSupplier( - spec.getIoConfig().getConsumerProperties(), + spec.getSpec().getIOConfig().getConsumerProperties(), sortingMapper, - spec.getIoConfig().getUri(), + spec.getSpec().getIOConfig().getUri(), taskTuningConfig.getRecordBufferSizeOrDefault(Runtime.getRuntime().maxMemory()), taskTuningConfig.getRecordBufferOfferTimeout(), taskTuningConfig.getMaxRecordsPerPollOrDefault() @@ -133,7 +133,7 @@ protected RecordSupplier setupRecordSupplier() @Override protected int getTaskGroupIdForPartition(String partitionId) { - return partitionId.hashCode() % spec.getIoConfig().getTaskCount(); + return partitionId.hashCode() % spec.getSpec().getIOConfig().getTaskCount(); } @Override @@ -158,7 +158,7 @@ protected SeekableStreamSupervisorReportPayload createReportPayloa int numPartitions, boolean includeOffsets) { - RabbitStreamSupervisorIOConfig ioConfig = spec.getIoConfig(); + RabbitStreamSupervisorIOConfig ioConfig = spec.getSpec().getIOConfig(); Map partitionLag = getRecordLagPerPartitionInLatestSequences(getHighestCurrentOffsets()); return new RabbitStreamSupervisorReportPayload( spec.getId(), @@ -423,7 +423,7 @@ protected String baseTaskName() @VisibleForTesting public RabbitStreamSupervisorIOConfig getIoConfig() { - return spec.getIoConfig(); + return spec.getSpec().getIOConfig(); } @VisibleForTesting 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 0f1e5cc88067..b638ed241221 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 @@ -130,17 +130,17 @@ public KafkaSupervisor( protected RecordSupplier setupRecordSupplier() { return new KafkaRecordSupplier( - spec.getIoConfig().getConsumerProperties(), + spec.getSpec().getIOConfig().getConsumerProperties(), sortingMapper, - spec.getIoConfig().getConfigOverrides(), - spec.getIoConfig().isMultiTopic() + spec.getSpec().getIOConfig().getConfigOverrides(), + spec.getSpec().getIOConfig().isMultiTopic() ); } @Override protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) { - Integer taskCount = spec.getIoConfig().getTaskCount(); + Integer taskCount = spec.getSpec().getIOConfig().getTaskCount(); if (spec.usePerpetuallyRunningTasks()) { int taskGroupId = getRangeBasedTaskGroupId(partitionId, taskCount); @@ -202,7 +202,7 @@ protected SeekableStreamSupervisorReportPayload creat boolean includeOffsets ) { - KafkaSupervisorIOConfig ioConfig = spec.getIoConfig(); + KafkaSupervisorIOConfig ioConfig = spec.getSpec().getIOConfig(); Map partitionLag = getRecordLagPerPartitionInLatestSequences(getHighestCurrentOffsets()); return new KafkaSupervisorReportPayload( spec.getId(), @@ -279,7 +279,7 @@ protected SeekableStreamIndexTaskIOConfig createUpdat SeekableStreamStartSequenceNumbers startSequenceNumbers = new SeekableStreamStartSequenceNumbers<>( - spec.getIoConfig().getStream(), + spec.getSpec().getIOConfig().getStream(), startingSequences, exclusiveStartSequenceNumberPartitions ); @@ -292,7 +292,7 @@ protected SeekableStreamIndexTaskIOConfig createUpdat SeekableStreamEndSequenceNumbers endSequenceNumbers = new SeekableStreamEndSequenceNumbers<>( - spec.getIoConfig().getStream(), + spec.getSpec().getIOConfig().getStream(), endingSequences ); @@ -309,15 +309,15 @@ protected SeekableStreamIndexTaskIOConfig createUpdat null, startSequenceNumbers, endSequenceNumbers, - spec.getIoConfig().getConsumerProperties(), - spec.getIoConfig().getPollTimeout(), + spec.getSpec().getIOConfig().getConsumerProperties(), + spec.getSpec().getIOConfig().getPollTimeout(), true, existingTaskGroup.getMinimumMessageTime(), existingTaskGroup.getMaximumMessageTime(), - spec.getIoConfig().getInputFormat(), - spec.getIoConfig().getConfigOverrides(), - spec.getIoConfig().isMultiTopic(), - spec.getIoConfig().getTaskDuration().getStandardMinutes() + spec.getSpec().getIOConfig().getInputFormat(), + spec.getSpec().getIOConfig().getConfigOverrides(), + spec.getSpec().getIOConfig().isMultiTopic(), + spec.getSpec().getIOConfig().getTaskDuration().getStandardMinutes() ); } @@ -647,7 +647,7 @@ protected String baseTaskName() @VisibleForTesting public KafkaSupervisorIOConfig getIoConfig() { - return spec.getIoConfig(); + return spec.getSpec().getIOConfig(); } @VisibleForTesting 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 9ab6a0d5791b..7175a58a57d8 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 @@ -133,8 +133,11 @@ import java.util.Map; import java.util.Properties; import java.util.TreeMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; @@ -233,10 +236,22 @@ public void setupTest() taskRunner = createMock(TaskRunner.class); indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); taskClient = createMock(SeekableStreamIndexTaskClient.class); + topic = getTopic(); + Map dummyOffsets = Map.of( + new KafkaTopicPartition(false, topic, 0), 100L, + new KafkaTopicPartition(false, topic, 1), 200L, + new KafkaTopicPartition(false, topic, 2), 300L + ); + EasyMock.expect(taskClient.pauseAndCheckpointAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(dummyOffsets)) + .anyTimes(); + EasyMock.expect(taskClient.updateConfigAsync( + EasyMock.anyString(), + EasyMock.anyObject() + )).andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.expect(taskClient.resumeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(true)).anyTimes(); taskQueue = createMock(TaskQueue.class); - - topic = getTopic(); topicPattern = getTopicPattern(); topicPostfix++; multiTopic = false; // assign to true in test if you wish to test multi-topic @@ -458,6 +473,209 @@ public SeekableStreamIndexTaskClient build( autoscaler.stop(); } + @Test + public void test_noInitialState_withPerpetuallyRunningTasks() throws Exception + { + final int taskCountMax = 2; + final int replicas = 1; + + // Synchronization mechanism for checkpoint coordination + final CountDownLatch checkpointTriggeredLatch = new CountDownLatch(1); + final AtomicReference checkpointTaskId = new AtomicReference<>(); + + KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( + null, + null + ) + { + @Override + public SeekableStreamIndexTaskClient build( + final String dataSource, + final TaskInfoProvider taskInfoProvider, + final SeekableStreamSupervisorTuningConfig tuningConfig, + final ScheduledExecutorService connectExec + ) + { + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout()); + Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries()); + return taskClient; + } + }; + + HashMap autoScalerConfig = new HashMap<>(); + autoScalerConfig.put("enableTaskAutoScaler", true); + autoScalerConfig.put("lagCollectionIntervalMillis", 500); + autoScalerConfig.put("lagCollectionRangeMillis", 500); + autoScalerConfig.put("scaleOutThreshold", 0); + autoScalerConfig.put("triggerScaleOutFractionThreshold", 0.0); + autoScalerConfig.put("scaleInThreshold", 1000000); + autoScalerConfig.put("triggerScaleInFractionThreshold", 0.8); + autoScalerConfig.put("scaleActionStartDelayMillis", 0); + autoScalerConfig.put("scaleActionPeriodMillis", 250); + autoScalerConfig.put("taskCountMax", taskCountMax); + autoScalerConfig.put("taskCountMin", 1); + autoScalerConfig.put("scaleInStep", 1); + autoScalerConfig.put("scaleOutStep", 2); + autoScalerConfig.put("minTriggerScaleActionFrequencyMillis", 1200000); + + final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); + consumerProperties.put("myCustomKey", "myCustomValue"); + consumerProperties.put("bootstrap.servers", kafkaHost); + + KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaIOConfigBuilder() + .withTopic(topic) + .withInputFormat(INPUT_FORMAT) + .withReplicas(replicas) + .withTaskCount(1) + .withConsumerProperties(consumerProperties) + .withAutoScalerConfig(OBJECT_MAPPER.convertValue(autoScalerConfig, LagBasedAutoScalerConfig.class)) + .withUseEarliestSequenceNumber(true) + .build(); + + final KafkaSupervisorTuningConfig tuningConfigOri = new KafkaTuningConfigBuilder() + .withIntermediatePersistPeriod(Period.years(1)) + .withResetOffsetAutomatically(false) + .withWorkerThreads(numThreads) + .withShutdownTimeout(TEST_SHUTDOWN_TIMEOUT) + .withMaxRowsInMemory(1000) + .withMaxRowsPerSegment(50000) + .withReportParseExceptions(false) + .withChatHandlerNumRetries(TEST_CHAT_RETRIES) + .withChatHandlerTimeout(TEST_HTTP_TIMEOUT.toStandardDuration()) + .build(); + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(kafkaSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(tuningConfigOri).anyTimes(); + EasyMock.replay(ingestionSchema); + + SeekableStreamSupervisorSpec testableSupervisorSpec = new KafkaSupervisorSpec( + null, + ingestionSchema, + dataSchema, + tuningConfigOri, + kafkaSupervisorIOConfig, + null, + false, + true, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + new SupervisorStateManagerConfig() + ); + + supervisor = new TestableKafkaSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + (KafkaSupervisorSpec) testableSupervisorSpec, + rowIngestionMetersFactory + ); + + SupervisorTaskAutoScaler autoscaler = testableSupervisorSpec.createAutoscaler(supervisor); + + + final KafkaSupervisorTuningConfig tuningConfig = supervisor.getTuningConfig(); + addSomeEvents(1); + + Capture captured = Capture.newInstance(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).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).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + int taskCountBeforeScale = supervisor.getIoConfig().getTaskCount(); + Assert.assertEquals(1, taskCountBeforeScale); + autoscaler.start(); + supervisor.runInternal(); + + // After task creation, simulate task discovery by updating the mock + // to return the newly created tasks so they can be adopted + Task createdTask = captured.getValue(); + Map createdTasks = Map.of( + createdTask.getId(), createdTask + ); + EasyMock.reset(taskQueue, taskMaster, taskClient, taskStorage); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)) + .andReturn(createdTasks).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); + taskQueue.update(EasyMock.anyObject()); + EasyMock.expectLastCall().anyTimes(); + EasyMock.expect(taskStorage.getStatus(createdTask.getId())) + .andReturn(Optional.of(TaskStatus.running(createdTask.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(createdTask.getId())) + .andReturn(Optional.of(createdTask)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(createdTask.getId())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)).anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(createdTask.getId())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.eq(createdTask.getId()), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(new TreeMap<>())).anyTimes(); + EasyMock.expect(taskClient.pauseAndCheckpointAsync(EasyMock.eq(createdTask.getId()))) + .andAnswer(() -> { + // Signal that checkpoint has been triggered by autoscaler and capture task ID + checkpointTaskId.set(createdTask.getId()); + checkpointTriggeredLatch.countDown(); + return Futures.immediateFuture(singlePartitionMap(topic, 0, 0L)); + }).anyTimes(); + EasyMock.expect(taskClient.pauseAsync(EasyMock.eq(createdTask.getId()))) + .andReturn(Futures.immediateFuture(singlePartitionMap(topic, 0, 0L))).anyTimes(); + EasyMock.expect(taskClient.setEndOffsetsAsync(EasyMock.eq(createdTask.getId()), EasyMock.anyObject(), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.expect(taskClient.updateConfigAsync(EasyMock.eq(createdTask.getId()), EasyMock.anyObject())) + .andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.expect(taskClient.resumeAsync(EasyMock.eq(createdTask.getId()))) + .andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.replay(taskQueue, taskMaster, taskClient, taskStorage); + + // Run again to discover and adopt the created tasks into activelyReadingTaskGroups + supervisor.runInternal(); + + // Wait for autoscaler to trigger pauseAndCheckpointAsync before proceeding with checkpoint + boolean checkpointTriggered = checkpointTriggeredLatch.await(10, TimeUnit.SECONDS); + Assert.assertTrue("Autoscaler should have triggered pauseAndCheckpointAsync within 10 seconds", checkpointTriggered); + + // Now simulate checkpoint handling - task runners hit checkpoint endpoints on overlord + // This is required for perpetually running tasks to complete the autoscaling process + String taskId = checkpointTaskId.get(); + Assert.assertNotNull("Task ID should be captured when pauseAndCheckpointAsync was called", taskId); + + Map checkpointOffsets = singlePartitionMap(topic, 0, 100L, 1, 200L, 2, 300L); + supervisor.checkpoint( + 0, // taskGroupId + new KafkaDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>(topic, checkpointOffsets, ImmutableSet.of()) + ) + ); + + // Allow some time for checkpoint processing and autoscaler to complete + Thread.sleep(2000); + verifyAll(); + + int taskCountAfterScale = supervisor.getIoConfig().getTaskCount(); + Assert.assertEquals(2, taskCountAfterScale); + + autoscaler.reset(); + autoscaler.stop(); + } + @Test public void testGetTaskRunnerType() throws JsonProcessingException { 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 7d8ce789384e..16c5899358b3 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 @@ -2003,9 +2003,6 @@ private List getCurrentParseErrors() } } - SeekableStreamIndexTaskTuningConfig ss = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()); - SeekableStreamSupervisorIOConfig oo = spec.getSpec().getIOConfig(); - // store a limited number of parse exceptions, keeping the most recent ones int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()).getMaxSavedParseExceptions() * spec.getSpec().getIOConfig().getTaskCount(); From 3392bdad711011c8ec3372fa772f93fa3a246127 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Tue, 23 Sep 2025 16:35:10 +0530 Subject: [PATCH 42/46] Address review comments related to Index task runner --- .../indexing/kafka/KafkaIndexTaskTest.java | 24 +- .../kinesis/KinesisIndexTaskTest.java | 18 +- ...eekableStreamIndexTaskClientAsyncImpl.java | 2 +- .../SeekableStreamIndexTaskRunner.java | 273 +++++++++--------- .../seekablestream/TaskConfigResponse.java | 75 +++++ .../common/OrderedSequenceNumber.java | 12 +- ...bleStreamIndexTaskClientAsyncImplTest.java | 8 +- 7 files changed, 238 insertions(+), 174 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 81a06a453bd0..9bfb80f190b3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -684,7 +684,7 @@ public void testIncrementalHandOff() throws Exception Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) || checkpoint2.getPartitionSequenceNumberMap() .equals(currentOffsets)); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(1, checkpointRequestsHash.size()); @@ -787,7 +787,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); while (task.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); @@ -806,7 +806,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), nextOffsets); - task.getRunner().setEndOffsets(nextOffsets, false); + task.getRunner().setEndOffsets(nextOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -915,7 +915,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(1, checkpointRequestsHash.size()); @@ -994,7 +994,7 @@ public void testCheckpointResetWithSameEndOffsets() throws Exception } long currentNextCheckpointTime = task.getRunner().getNextCheckpointTime(); final Map nextEndOffsets = task.getRunner().getLastSequenceMetadata().getStartOffsets(); - task.getRunner().setEndOffsets(nextEndOffsets, false); + task.getRunner().setEndOffsets(nextEndOffsets, false, true); long newNextCheckpointTime = task.getRunner().getNextCheckpointTime(); Assert.assertTrue( StringUtils.format( @@ -1089,8 +1089,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception Map currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, false); - staleReplica.getRunner().setEndOffsets(currentOffsets, false); + normalReplica.getRunner().setEndOffsets(currentOffsets, false, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, false, true); while (normalReplica.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); @@ -1103,8 +1103,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception currentOffsets = ImmutableMap.copyOf(staleReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, true); - staleReplica.getRunner().setEndOffsets(currentOffsets, true); + normalReplica.getRunner().setEndOffsets(currentOffsets, true, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, true, true); Assert.assertEquals(TaskState.SUCCESS, normalReplicaFuture.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, staleReplicaFuture.get().getStatusCode()); @@ -2191,7 +2191,7 @@ public void testRestoreAfterPersistingSequences() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); // Set endOffsets to persist sequences - task1.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L), false); + task1.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L), false, true); // Stop without publishing segment task1.stopGracefully(toolboxFactory.build(task1).getConfig()); @@ -2485,12 +2485,12 @@ public void testRunWithDuplicateRequest() throws Exception // first setEndOffsets request task.getRunner().pause(); - task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true); + task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true, true); Assert.assertEquals(Status.READING, task.getRunner().getStatus()); // duplicate setEndOffsets request task.getRunner().pause(); - task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true); + task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true, true); Assert.assertEquals(Status.READING, task.getRunner().getStatus()); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 98dfea4333cc..fd57a9de3016 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -612,7 +612,7 @@ public void testIncrementalHandOff() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -696,7 +696,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets1, currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); waitUntil(task, this::isTaskPaused); @@ -704,7 +704,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(checkpointOffsets2, nextOffsets); - task.getRunner().setEndOffsets(nextOffsets, false); + task.getRunner().setEndOffsets(nextOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -1753,7 +1753,7 @@ public void testRestoreAfterPersistingSequences() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets1, currentOffsets); - task1.getRunner().setEndOffsets(currentOffsets, false); + task1.getRunner().setEndOffsets(currentOffsets, false, true); // Stop without publishing segment task1.stopGracefully(toolboxFactory.build(task1).getConfig()); @@ -1874,7 +1874,7 @@ public void testRunWithPauseAndResume() throws Exception replayAll(); - task.getRunner().setEndOffsets(currentOffsets, true); + task.getRunner().setEndOffsets(currentOffsets, true, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -2033,8 +2033,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception Map currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets1, currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, false); - staleReplica.getRunner().setEndOffsets(currentOffsets, false); + normalReplica.getRunner().setEndOffsets(currentOffsets, false, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, false, true); waitUntil(normalReplica, this::isTaskPaused); waitUntil(staleReplica, this::isTaskPaused); @@ -2044,8 +2044,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception currentOffsets = ImmutableMap.copyOf(staleReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets2, currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, true); - staleReplica.getRunner().setEndOffsets(currentOffsets, true); + normalReplica.getRunner().setEndOffsets(currentOffsets, true, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, true, true); Assert.assertEquals(TaskState.SUCCESS, normalReplicaFuture.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, staleReplicaFuture.get().getStatusCode()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 60f8669e7eef..99320af74b56 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -213,7 +213,7 @@ public ListenableFuture registerNewVersionOfPendingSegmentAsync( @Override public ListenableFuture updateConfigAsync(String taskId, TaskConfigUpdateRequest updateRequest) { - final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/updateConfig") + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/config") .jsonContent(jsonMapper, updateRequest); return makeRequest(taskId, requestBuilder) .handler(IgnoreHttpResponseHandler.INSTANCE) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 056c9a3fe3e9..b876e1ee161b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -188,6 +188,7 @@ public enum Status // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, // [status] is changed to STARTING and [shouldResume] is signalled. private final Lock pauseLock = new ReentrantLock(); + private final Lock updateConfigLock = new ReentrantLock(); private final Condition hasPaused = pauseLock.newCondition(); private final Condition shouldResume = pauseLock.newCondition(); @@ -216,7 +217,7 @@ public enum Status private final InputFormat inputFormat; @Nullable private final InputRowParser parser; - private String stream; + private final String stream; private final Set publishingSequences = Sets.newConcurrentHashSet(); private final Set publishedSequences = Sets.newConcurrentHashSet(); @@ -254,7 +255,7 @@ public enum Status private volatile DateTime minMessageTime; private volatile DateTime maxMessageTime; private final ScheduledExecutorService rejectionPeriodUpdaterExec; - private AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); + private final AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); public SeekableStreamIndexTaskRunner( @@ -605,16 +606,15 @@ public void run() // restart publishing of sequences (if any) maybePersistAndPublishSequences(committerSupplier); - assignment = assignPartitions(this.recordSupplier); - possiblyResetDataSourceMetadata(toolbox, this.recordSupplier, assignment); - seekToStartingSequence(this.recordSupplier, assignment); + assignment = assignPartitions(recordSupplier); + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); + seekToStartingSequence(recordSupplier, assignment); ingestionState = IngestionState.BUILD_SEGMENTS; // Main loop. // Could eventually support leader/follower mode (for keeping replicas more in sync) - log.info("Task perpetuallyRunning: %s", task.isPerpetuallyRunning()); - boolean stillReading = !assignment.isEmpty() || task.isPerpetuallyRunning(); + boolean stillReading = isStillReading(); status = Status.READING; Throwable caughtExceptionInner = null; @@ -624,15 +624,13 @@ public void run() // The partition assignments may have changed while paused by a call to setEndOffsets() so reassign // partitions upon resuming. Don't call "seekToStartingSequence" after "assignPartitions", because there's // no need to re-seek here. All we're going to be doing is dropping partitions. - assignment = assignPartitions(this.recordSupplier); - possiblyResetDataSourceMetadata(toolbox, this.recordSupplier, assignment); - - if (assignment.isEmpty()) { - if (!task.isPerpetuallyRunning()) { - log.debug("All partitions have been fully read."); - publishOnStop.set(true); - stopRequested.set(true); - } + assignment = assignPartitions(recordSupplier); + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); + + if (assignment.isEmpty() && !task.isPerpetuallyRunning()) { + log.debug("All partitions have been fully read."); + publishOnStop.set(true); + stopRequested.set(true); } } @@ -656,12 +654,12 @@ public void run() // calling getRecord() ensures that exceptions specific to kafka/kinesis like OffsetOutOfRangeException // are handled in the subclasses. List> records = getRecords( - this.recordSupplier, + recordSupplier, toolbox ); // note: getRecords() also updates assignment - stillReading = !assignment.isEmpty() || task.isPerpetuallyRunning(); + stillReading = isStillReading(); SequenceMetadata sequenceToCheckpoint = null; AppenderatorDriverAddResult pushTriggeringAddResult = null; @@ -765,8 +763,8 @@ public void onFailure(Throwable t) if (!moreToReadAfterThisRecord && assignment.remove(record.getStreamPartition())) { log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); - this.recordSupplier.assign(assignment); - stillReading = !assignment.isEmpty() || task.isPerpetuallyRunning(); + recordSupplier.assign(assignment); + stillReading = isStillReading(); } } @@ -857,17 +855,67 @@ public void onFailure(Throwable t) // We need to copy sequences here, because the success callback in publishAndRegisterHandoff removes items from // the sequence list. If a publish finishes before we finish iterating through the sequence list, we can // end up skipping some sequences. - populateSequencesToPublish(); + List> sequencesSnapshot = new ArrayList<>(sequences); + for (int i = 0; i < sequencesSnapshot.size(); i++) { + final SequenceMetadata sequenceMetadata = sequencesSnapshot.get(i); + if (!publishingSequences.contains(sequenceMetadata.getSequenceName()) + && !publishedSequences.contains(sequenceMetadata.getSequenceName())) { + final boolean isLast = i == (sequencesSnapshot.size() - 1); + if (isLast) { + // Shorten endOffsets of the last sequence to match currOffsets. + sequenceMetadata.setEndOffsets(currOffsets); + } + + // Update assignments of the sequence, which should clear them. (This will be checked later, when the + // Committer is built.) + sequenceMetadata.updateAssignments(currOffsets, this::isMoreToReadAfterReadingRecord); + publishingSequences.add(sequenceMetadata.getSequenceName()); + // persist already done in finally, so directly add to publishQueue + publishAndRegisterHandoff(sequenceMetadata); + } + } if (backgroundThreadException != null) { throw new RuntimeException(backgroundThreadException); } - // Wait for publish futures to complete if it's a standard task. - if (!task.isPerpetuallyRunning()) { - handoffWaitMs = waitForPublishAndHandoffCompletion(); + // Wait for publish futures to complete. + Futures.allAsList(publishWaitList).get(); + + // Wait for handoff futures to complete. + // Note that every publishing task (created by calling AppenderatorDriver.publish()) has a corresponding + // handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it + // failed to persist sequences. It might also return null if handoff failed, but was recoverable. + // See publishAndRegisterHandoff() for details. + List handedOffList = Collections.emptyList(); + ingestionState = IngestionState.SEGMENT_AVAILABILITY_WAIT; + if (tuningConfig.getHandoffConditionTimeout() == 0) { + handedOffList = Futures.allAsList(handOffWaitList).get(); + } else { + final long start = System.nanoTime(); + try { + handedOffList = Futures.allAsList(handOffWaitList) + .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); + } + catch (TimeoutException e) { + // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception + // here. + log.makeAlert("Timeout waiting for handoff") + .addData("taskId", task.getId()) + .addData("handoffConditionTimeout", tuningConfig.getHandoffConditionTimeout()) + .emit(); + } + finally { + handoffWaitMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + } } + for (SegmentsAndCommitMetadata handedOff : handedOffList) { + log.info( + "Handoff complete for segments: %s", + String.join(", ", Lists.transform(handedOff.getSegments(), DataSegment::toString)) + ); + } appenderator.close(); } catch (InterruptedException | RejectedExecutionException e) { @@ -939,69 +987,9 @@ public void onFailure(Throwable t) return TaskStatus.success(task.getId()); } - private long waitForPublishAndHandoffCompletion() throws ExecutionException, InterruptedException + private boolean isStillReading() { - long handoffWaitMs = 0L; - Futures.allAsList(publishWaitList).get(); - - // Wait for handoff futures to complete. - // Note that every publishing task (created by calling AppenderatorDriver.publish()) has a corresponding - // handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it - // failed to persist sequences. It might also return null if handoff failed, but was recoverable. - // See publishAndRegisterHandoff() for details. - List handedOffList = Collections.emptyList(); - ingestionState = IngestionState.SEGMENT_AVAILABILITY_WAIT; - if (tuningConfig.getHandoffConditionTimeout() == 0) { - handedOffList = Futures.allAsList(handOffWaitList).get(); - } else { - final long start = System.nanoTime(); - try { - handedOffList = Futures.allAsList(handOffWaitList) - .get(tuningConfig.getHandoffConditionTimeout(), TimeUnit.MILLISECONDS); - } - catch (TimeoutException e) { - // Handoff timeout is not an indexing failure, but coordination failure. We simply ignore timeout exception - // here. - log.makeAlert("Timeout waiting for handoff") - .addData("taskId", task.getId()) - .addData("handoffConditionTimeout", tuningConfig.getHandoffConditionTimeout()) - .emit(); - } - finally { - handoffWaitMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); - } - } - - for (SegmentsAndCommitMetadata handedOff : handedOffList) { - log.info( - "Handoff complete for segments: %s", - String.join(", ", Lists.transform(handedOff.getSegments(), DataSegment::toString)) - ); - } - return handoffWaitMs; - } - - private void populateSequencesToPublish() - { - List> sequencesSnapshot = new ArrayList<>(sequences); - for (int i = 0; i < sequencesSnapshot.size(); i++) { - final SequenceMetadata sequenceMetadata = sequencesSnapshot.get(i); - if (!publishingSequences.contains(sequenceMetadata.getSequenceName()) - && !publishedSequences.contains(sequenceMetadata.getSequenceName())) { - final boolean isLast = i == (sequencesSnapshot.size() - 1); - if (isLast) { - // Shorten endOffsets of the last sequence to match currOffsets. - sequenceMetadata.setEndOffsets(currOffsets); - } - - // Update assignments of the sequence, which should clear them. (This will be checked later, when the - // Committer is built.) - sequenceMetadata.updateAssignments(currOffsets, this::isMoreToReadAfterReadingRecord); - publishingSequences.add(sequenceMetadata.getSequenceName()); - // persist already done in finally, so directly add to publishQueue - publishAndRegisterHandoff(sequenceMetadata); - } - } + return !assignment.isEmpty() || task.isPerpetuallyRunning(); } private TaskLockType getTaskLockType() @@ -1626,10 +1614,10 @@ public Map getEndOffsets() @GET @Path("/config") @Produces(MediaType.APPLICATION_JSON) - public SeekableStreamIndexTaskIOConfig getIOConfigHTTP(@Context final HttpServletRequest req) + public TaskConfigResponse getConfigHTTP(@Context final HttpServletRequest req) { authorizationCheck(req); - return ioConfig; + return new TaskConfigResponse(ioConfig); } @POST @@ -1644,7 +1632,7 @@ public Response setEndOffsetsHTTP( ) throws InterruptedException { authorizationCheck(req); - return setEndOffsets(sequences, finish); + return setEndOffsets(sequences, finish, !waitForConfigUpdate.get()); } @POST @@ -1750,47 +1738,17 @@ public Response getUnparseableEvents( } @POST - @Path("/updateConfig") + @Path("/config") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response updateConfig(TaskConfigUpdateRequest request, @Context final HttpServletRequest req) - throws InterruptedException { authorizationCheck(req); if (!waitForConfigUpdate.get()) { - return Response.status(409).entity("Task must be paused for checkpoint completion before updating config").build(); + return Response.status(409).entity("Task must have been paused and checkpointed before updating config.").build(); } try { - log.info("Attempting to update config to [%s]", request.getIoConfig()); - - SeekableStreamIndexTaskIOConfig newIoConfig = (SeekableStreamIndexTaskIOConfig) - toolbox.getJsonMapper().convertValue(request.getIoConfig(), SeekableStreamIndexTaskIOConfig.class); - setIOConfig(newIoConfig); - createNewSequenceFromIoConfig(newIoConfig); - - assignment = assignPartitions(recordSupplier); - boolean shouldResume = true; - if (!assignment.isEmpty()) { - possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); - seekToStartingSequence(recordSupplier, assignment); - } else { - // if there is no assignment, It means that there was no partition assigned to this task after scaling down. - pause(); - shouldResume = false; - } - - log.info("Config updated to [%s]", this.ioConfig); - toolbox.getEmitter().emit(ServiceMetricEvent.builder() - .setDimension(DruidMetrics.TASK_ID, task.getId()) - .setDimension(DruidMetrics.TASK_TYPE, task.getType()) - .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) - .setMetric("task/config/update/success", 1) - .build(ImmutableMap.of())); - if (shouldResume) { - resume(); - } - waitForConfigUpdate.set(false); - return Response.ok().build(); + return updateTaskRunnerConfig(request); } catch (Exception e) { log.makeAlert(e, "Failed to update task config"); @@ -1799,12 +1757,41 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt } } + /** + * Updates the task's ioConfig, creates a new sequence from the new ioConfig, re-assigns partitions and seeks to + * the new starting offsets. If there is no partition assigned to this task due to a scale down, the task will be + * paused. + */ + private Response updateTaskRunnerConfig(TaskConfigUpdateRequest request) throws IOException, InterruptedException + { + log.info("Attempting to update config to [%s]", request.getIoConfig()); + SeekableStreamIndexTaskIOConfig newIoConfig = request.getIoConfig(); + setIOConfig(newIoConfig); + createNewSequenceFromIoConfig(newIoConfig); + + assignment = assignPartitions(recordSupplier); + if (!assignment.isEmpty()) { + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); + seekToStartingSequence(recordSupplier, assignment); + resume(); + } + + log.info("Config updated to [%s]", this.ioConfig); + toolbox.getEmitter().emit(ServiceMetricEvent.builder() + .setDimension(DruidMetrics.TASK_ID, task.getId()) + .setDimension(DruidMetrics.TASK_TYPE, task.getType()) + .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) + .setMetric("task/config/update/success", 1) + .build(ImmutableMap.of())); + waitForConfigUpdate.set(false); + return Response.ok().build(); + } + private void setIOConfig( SeekableStreamIndexTaskIOConfig ioConfig ) { this.ioConfig = ioConfig; - this.stream = ioConfig.getStartSequenceNumbers().getStream(); this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); this.minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); this.maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); @@ -1857,7 +1844,7 @@ private void checkpointSequences() try { final SequenceMetadata latestSequence = getLastSequenceMetadata(); if (!latestSequence.isCheckpointed()) { - final CheckPointDataSourceMetadataAction checkpointAciton = new CheckPointDataSourceMetadataAction( + final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction( getSupervisorId(), ioConfig.getTaskGroupId(), null, @@ -1869,7 +1856,9 @@ private void checkpointSequences() ) ) ); - toolbox.getTaskActionClient().submit(checkpointAciton); + if (!toolbox.getTaskActionClient().submit(checkpointAction)) { + throw new ISE("Checkpoint request with sequences [%s] failed, dying", currOffsets); + } } } catch (Exception e) { @@ -1881,7 +1870,8 @@ private void checkpointSequences() @VisibleForTesting public Response setEndOffsets( Map sequenceNumbers, - boolean finish // this field is only for internal purposes, shouldn't be usually set by users + boolean finish, // this field is only for internal purposes, shouldn't be usually set by users + boolean shouldPause ) throws InterruptedException { if (sequenceNumbers == null) { @@ -1898,11 +1888,9 @@ public Response setEndOffsets( ) .build(); } else { - try { + try { // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock - if (!waitForConfigUpdate.get()) { - pauseLock.lockInterruptibly(); - } + pauseLock.lockInterruptibly(); // Perform all sequence related checks before checking for isPaused() // and after acquiring pauseLock to correctly guard against duplicate requests Preconditions.checkState(sequenceNumbers.size() > 0, "No sequences found to set end sequences"); @@ -1926,7 +1914,7 @@ public Response setEndOffsets( || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resetNextCheckpointTime(); - if (!waitForConfigUpdate.get()) { + if (shouldPause) { resume(); } return Response.ok(sequenceNumbers).build(); @@ -2005,13 +1993,11 @@ public Response setEndOffsets( .build(); } finally { - if (!waitForConfigUpdate.get()) { - pauseLock.unlock(); - } + pauseLock.unlock(); } } - if (!waitForConfigUpdate.get()) { + if (shouldPause) { resume(); } @@ -2072,15 +2058,24 @@ public Response pauseHTTP( @Produces(MediaType.APPLICATION_JSON) public Response pauseAndCheckpointHTTP( @Context final HttpServletRequest req - ) throws InterruptedException, JsonProcessingException + ) throws InterruptedException { authorizationCheck(req); if (!waitForConfigUpdate.compareAndSet(false, true)) { return Response.ok().entity("Task is already paused for checkpoint completion").build(); } - pause(); - checkpointSequences(); - return Response.ok().entity(toolbox.getJsonMapper().writeValueAsString(getCurrentOffsets())).build(); + Response pauseResponse = pause(); + if (pauseResponse.getStatus() == 409) { + waitForConfigUpdate.set(false); + return pauseResponse; + } + try { + checkpointSequences(); + } catch (Exception e) { + waitForConfigUpdate.set(false); + resume(); + } + return Response.ok().entity(getCurrentOffsets()).build(); } @VisibleForTesting diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java new file mode 100644 index 000000000000..6b6c86e5dbd5 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java @@ -0,0 +1,75 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Response object for returning the configuration of a running {@link SeekableStreamIndexTask}. + */ +public class TaskConfigResponse +{ + private final SeekableStreamIndexTaskIOConfig ioConfig; + + @JsonCreator + public TaskConfigResponse( + @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig + ) + { + this.ioConfig = ioConfig; + } + + @JsonProperty + public SeekableStreamIndexTaskIOConfig getIoConfig() + { + return ioConfig; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TaskConfigResponse that = (TaskConfigResponse) o; + return Objects.equals(ioConfig, that.ioConfig); + } + + @Override + public int hashCode() + { + return Objects.hash(ioConfig); + } + + @Override + public String toString() + { + return "TaskConfigUpdateRequest{" + + "ioConfig=" + ioConfig + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java index 870daaa17358..24fe811c06f1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java @@ -100,16 +100,10 @@ public boolean isMoreToReadBeforeReadingRecord(OrderedSequenceNumber Date: Wed, 24 Sep 2025 17:21:04 +0530 Subject: [PATCH 43/46] Stop persisting task configs and address other comments --- .../rabbitstream/RabbitStreamIndexTask.java | 1 + .../druid/indexing/kafka/KafkaIndexTask.java | 5 +- .../kafka/supervisor/KafkaSupervisor.java | 5 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 6 +- .../indexing/kafka/KafkaIndexTaskTest.java | 1 + .../kafka/supervisor/KafkaSupervisorTest.java | 1 + .../indexing/kinesis/KinesisIndexTask.java | 1 + .../overlord/HeapMemoryTaskStorage.java | 11 -- .../overlord/MetadataTaskStorage.java | 23 ---- .../druid/indexing/overlord/TaskQueue.java | 40 ------ .../druid/indexing/overlord/TaskStorage.java | 7 -- .../SeekableStreamIndexTask.java | 11 +- .../SeekableStreamIndexTaskClient.java | 9 +- ...eekableStreamIndexTaskClientAsyncImpl.java | 15 ++- .../SeekableStreamIndexTaskRunner.java | 20 ++- .../seekablestream/TaskConfigResponse.java | 49 +++++--- .../TaskConfigUpdateRequest.java | 35 ++++-- .../supervisor/SeekableStreamSupervisor.java | 90 ++++++-------- .../SeekableStreamSupervisorSpec.java | 25 +++- .../overlord/HeapMemoryTaskStorageTest.java | 19 --- .../MetadataTaskStorageUpdateTaskTest.java | 117 ------------------ .../indexing/overlord/TaskQueueTest.java | 30 ----- ...bleStreamIndexTaskClientAsyncImplTest.java | 8 +- ...SeekableStreamIndexTaskRunnerAuthTest.java | 2 +- .../SeekableStreamSupervisorSpecTest.java | 26 ++-- .../TaskConfigUpdateRequestTest.java | 7 +- .../common/OrderedSequenceNumberTest.java | 12 +- .../SeekableStreamSupervisorStateTest.java | 63 +++++----- .../overlord/supervisor/SupervisorSpec.java | 6 + .../SQLMetadataSupervisorManager.java | 3 +- .../org/apache/druid/cli/CliPeonTest.java | 2 +- 31 files changed, 244 insertions(+), 406 deletions(-) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java index 85bf39cef1c1..4fe8cdae2ab8 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java @@ -69,6 +69,7 @@ public RabbitStreamIndexTask( ioConfig, context, getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + null, null ); this.configMapper = configMapper; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index c9904239b833..6f77b99434cd 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -71,6 +71,7 @@ public KafkaIndexTask( @JsonProperty("ioConfig") KafkaIndexTaskIOConfig ioConfig, @JsonProperty("context") Map context, @JsonProperty("isPerpetuallyRunning") @Nullable Boolean isPerpetuallyRunning, + @JsonProperty("supervisorSpecVersion") @Nullable String supervisorSpecVersion, @JacksonInject ObjectMapper configMapper ) { @@ -83,7 +84,8 @@ public KafkaIndexTask( ioConfig, context, getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), - isPerpetuallyRunning + isPerpetuallyRunning, + supervisorSpecVersion ); this.configMapper = configMapper; @@ -147,6 +149,7 @@ protected KafkaRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) (KafkaIndexTaskIOConfig) newIoConfig, getContext(), isPerpetuallyRunning(), + getSupervisorSpecVersion(), configMapper ); } 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 b638ed241221..32f6bc09b99e 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 @@ -142,7 +142,7 @@ protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) { Integer taskCount = spec.getSpec().getIOConfig().getTaskCount(); - if (spec.usePerpetuallyRunningTasks()) { + if (spec.usePersistentTasks()) { int taskGroupId = getRangeBasedTaskGroupId(partitionId, taskCount); log.debug("Range-based assignment for partition [%s]: taskGroupId [%d] when taskCount is [%d]", partitionId, taskGroupId, taskCount); return taskGroupId; @@ -347,7 +347,8 @@ protected List context, @JsonProperty("suspended") Boolean suspended, - @JsonProperty("usePerpetuallyRunningTasks") @Nullable Boolean usePerpetuallyRunningTasks, + @JsonProperty("usePersistentTasks") @Nullable Boolean usePersistentTasks, @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, @@ -91,7 +91,7 @@ public KafkaSupervisorSpec( monitorSchedulerConfig, rowIngestionMetersFactory, supervisorStateManagerConfig, - usePerpetuallyRunningTasks + usePersistentTasks ); } @@ -163,7 +163,7 @@ protected KafkaSupervisorSpec toggleSuspend(boolean suspend) getIoConfig(), getContext(), suspend, - usePerpetuallyRunningTasks(), + usePersistentTasks(), taskStorage, taskMaster, indexerMetadataStorageCoordinator, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 9bfb80f190b3..c48f09b2c983 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2932,6 +2932,7 @@ private KafkaIndexTask createTask( ioConfig, context, null, + "v1", OBJECT_MAPPER ); task.setPollRetryMs(POLL_RETRY_MS); 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 7175a58a57d8..9449d39d41ae 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 @@ -5869,6 +5869,7 @@ private KafkaIndexTask createKafkaIndexTask( ), Collections.emptyMap(), null, + "v1", OBJECT_MAPPER ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index d7ff577f03b7..3f9a229d6ce6 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -85,6 +85,7 @@ public KinesisIndexTask( ioConfig, context, getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + null, null ); this.useListShards = useListShards; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java index fdee190d545b..b3201ea9314f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -304,17 +304,6 @@ public List getLocks(final String taskid) } } - @Override - public void updateTask(Task task) - { - final Optional status = getStatus(task.getId()); - if (!status.isPresent()) { - throw new IllegalStateException("No task found for id [" + task.getId() + "]"); - } - final TaskInfo updatedTaskInfo = new TaskInfo(DateTimes.nowUtc(), status.get(), task); - tasks.put(task.getId(), updatedTaskInfo); - } - @Override public void removeTasksOlderThan(final long timestamp) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java index 55e8215876c1..9dba47c62780 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java @@ -296,27 +296,4 @@ private Map getLocksWithIds(final String taskid) { return handler.getLocks(taskid); } - - @Override - public void updateTask(Task task) - { - Preconditions.checkNotNull(task, "task"); - log.info("Updating task [%s].", task.getId()); - final String taskId = task.getId(); - - Optional existingTask = getTask(taskId); - if (!existingTask.isPresent()) { - throw new ISE("No task found for id [%s]", taskId); - } - - try { - handler.update(taskId, task); - } - catch (DruidException e) { - throw e; - } - catch (Exception e) { - throw new RuntimeException(e); - } - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 1a01ae1e118a..0cfe481a8fea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -555,46 +555,6 @@ public boolean add(final Task task) } } - public void update(final Task task) - { - IdUtils.validateId("Task ID", task.getId()); - - if (!taskStorage.getTask(task.getId()).isPresent()) { - throw new ISE("Task[%s] does not exist in storage", task.getId()); - } - - validateTaskPayload(task); - startStopLock.readLock().lock(); - - try { - Preconditions.checkState(active, "Queue is not active!"); - Preconditions.checkNotNull(task, "task"); - - final String taskId = task.getId(); - if (!activeTasks.containsKey(taskId)) { - throw new ISE("Task[%s] is not in the queue", taskId); - } - - updateTaskEntry(taskId, entry -> { - // Create new TaskInfo with updated task while preserving created time and status - final TaskInfo existingTaskInfo = entry.taskInfo; - entry.taskInfo = new TaskInfo( - existingTaskInfo.getCreatedTime(), - existingTaskInfo.getStatus(), - task - ); - log.info("Updated task [%s] in queue hashmap", taskId); - }); - - taskStorage.updateTask(task); - log.info("Updated task [%s] in storage", taskId); - requestManagement(); - } - finally { - startStopLock.readLock().unlock(); - } - } - @GuardedBy("startStopLock") private void addTaskInternal(final TaskInfo taskInfo, final DateTime updateTime) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java index f1b9c92cedfe..49a248ec4b8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java @@ -182,11 +182,4 @@ default List getTaskInfos( * @return list of TaskLocks for the given task */ List getLocks(String taskid); - - /** - * Updates an existing task with new configuration. - * This is used for updating task payload (e.g., IOConfig changes for perpetual tasks) - * while preserving the task status and other metadata. - */ - void updateTask(Task task); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 683b43704749..43c0386c34ca 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -72,6 +72,7 @@ public abstract class SeekableStreamIndexTask ioConfig, @Nullable final Map context, @Nullable final String groupId, - @Nullable final Boolean isPerpetuallyRunning + @Nullable final Boolean isPerpetuallyRunning, + @Nullable final String supervisorSpecVersion ) { super( @@ -109,6 +111,7 @@ public SeekableStreamIndexTask( this.lockTypeToUse = TaskLocks.determineLockTypeForAppend(getContext()); this.supervisorId = Preconditions.checkNotNull(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), "supervisorId"); this.isPerpetuallyRunning = Configs.valueOrDefault(isPerpetuallyRunning, false); + this.supervisorSpecVersion = Configs.valueOrDefault(supervisorSpecVersion, ""); } protected static String getFormattedGroupId(String supervisorId, String type) @@ -162,6 +165,12 @@ public boolean isPerpetuallyRunning() return isPerpetuallyRunning; } + @JsonProperty("supervisorSpecVersion") + public String getSupervisorSpecVersion() + { + return supervisorSpecVersion; + } + @Override public TaskStatus runTask(final TaskToolbox toolbox) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 3bdcf91004a2..e35ec1b61fa4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -175,7 +175,14 @@ ListenableFuture registerNewVersionOfPendingSegmentAsync( * * Task-side is {@link SeekableStreamIndexTaskRunner#updateConfig}. */ - ListenableFuture updateConfigAsync(String taskId, TaskConfigUpdateRequest updateRequest); + ListenableFuture updateConfigAsync(String taskId, TaskConfigUpdateRequest updateRequest); + + /** + * Gets the current configuration of a running task. + * + * Task-side is {@link SeekableStreamIndexTaskRunner#getConfigHTTP}. + */ + ListenableFuture> getTaskConfigAsync(String taskId); Class getPartitionType(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 99320af74b56..693f7cf585bf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -211,7 +211,10 @@ public ListenableFuture registerNewVersionOfPendingSegmentAsync( } @Override - public ListenableFuture updateConfigAsync(String taskId, TaskConfigUpdateRequest updateRequest) + public ListenableFuture updateConfigAsync( + String taskId, + TaskConfigUpdateRequest updateRequest + ) { final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/config") .jsonContent(jsonMapper, updateRequest); @@ -236,6 +239,16 @@ public ListenableFuture updateConfigAsync(String taskId, TaskConfigUpda .go(); } + @Override + public ListenableFuture> getTaskConfigAsync(String taskId) + { + return makeRequest(taskId, new RequestBuilder(HttpMethod.GET, "/config")) + .handler(new BytesFullResponseHandler()) + .onSuccess(r -> (TaskConfigResponse) JacksonUtils.readValue(jsonMapper, r.getContent(), TaskConfigResponse.class)) + .onNotAvailable(e -> Either.value(null)) + .go(); + } + @Override public ListenableFuture setEndOffsetsAsync( final String id, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index b876e1ee161b..016e8c71060c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -211,6 +211,7 @@ public enum Status private volatile Set> assignment; private volatile RecordSupplier recordSupplier; private SeekableStreamIndexTaskIOConfig ioConfig; + private volatile String supervisorSpecVersion; private final SeekableStreamIndexTaskTuningConfig tuningConfig; private final InputRowSchema inputRowSchema; @Nullable @@ -276,6 +277,7 @@ public SeekableStreamIndexTaskRunner( this.sequences = new CopyOnWriteArrayList<>(); this.ingestionState = IngestionState.NOT_STARTED; this.lockGranularityToUse = lockGranularityToUse; + this.supervisorSpecVersion = task.getSupervisorSpecVersion(); minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); @@ -1614,10 +1616,10 @@ public Map getEndOffsets() @GET @Path("/config") @Produces(MediaType.APPLICATION_JSON) - public TaskConfigResponse getConfigHTTP(@Context final HttpServletRequest req) + public TaskConfigResponse getConfigHTTP(@Context final HttpServletRequest req) { authorizationCheck(req); - return new TaskConfigResponse(ioConfig); + return new TaskConfigResponse<>(ioConfig, supervisorSpecVersion); } @POST @@ -1741,7 +1743,10 @@ public Response getUnparseableEvents( @Path("/config") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public Response updateConfig(TaskConfigUpdateRequest request, @Context final HttpServletRequest req) + public Response updateConfig( + TaskConfigUpdateRequest request, + @Context final HttpServletRequest req + ) { authorizationCheck(req); if (!waitForConfigUpdate.get()) { @@ -1762,12 +1767,14 @@ public Response updateConfig(TaskConfigUpdateRequest request, @Context final Htt * the new starting offsets. If there is no partition assigned to this task due to a scale down, the task will be * paused. */ - private Response updateTaskRunnerConfig(TaskConfigUpdateRequest request) throws IOException, InterruptedException + private Response updateTaskRunnerConfig(TaskConfigUpdateRequest request) + throws IOException, InterruptedException { log.info("Attempting to update config to [%s]", request.getIoConfig()); SeekableStreamIndexTaskIOConfig newIoConfig = request.getIoConfig(); setIOConfig(newIoConfig); createNewSequenceFromIoConfig(newIoConfig); + supervisorSpecVersion = request.getSupervisorSpecVersion(); assignment = assignPartitions(recordSupplier); if (!assignment.isEmpty()) { @@ -1888,7 +1895,7 @@ public Response setEndOffsets( ) .build(); } else { - try { + try { // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock pauseLock.lockInterruptibly(); // Perform all sequence related checks before checking for isPaused() @@ -2071,7 +2078,8 @@ public Response pauseAndCheckpointHTTP( } try { checkpointSequences(); - } catch (Exception e) { + } + catch (Exception e) { waitForConfigUpdate.set(false); resume(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java index 6b6c86e5dbd5..62af177414d0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java @@ -28,48 +28,59 @@ /** * Response object for returning the configuration of a running {@link SeekableStreamIndexTask}. */ -public class TaskConfigResponse +public class TaskConfigResponse { - private final SeekableStreamIndexTaskIOConfig ioConfig; + private final SeekableStreamIndexTaskIOConfig ioConfig; + private final String supervisorSpecVersion; @JsonCreator public TaskConfigResponse( - @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig + @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig, + @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion ) { this.ioConfig = ioConfig; + this.supervisorSpecVersion = supervisorSpecVersion; } @JsonProperty - public SeekableStreamIndexTaskIOConfig getIoConfig() + public SeekableStreamIndexTaskIOConfig getIoConfig() { return ioConfig; } + @JsonProperty + public String getSupervisorSpecVersion() + { + return supervisorSpecVersion; + } + @Override - public boolean equals(Object o) + public String toString() { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TaskConfigResponse that = (TaskConfigResponse) o; - return Objects.equals(ioConfig, that.ioConfig); + return "TaskConfigResponse{" + + "ioConfig=" + ioConfig + + ", supervisorSpecVersion='" + supervisorSpecVersion + '\'' + + '}'; } @Override - public int hashCode() + public boolean equals(Object object) { - return Objects.hash(ioConfig); + if (object == null || getClass() != object.getClass()) { + return false; + } + TaskConfigResponse that = (TaskConfigResponse) object; + return Objects.equals(ioConfig, that.ioConfig) && Objects.equals( + supervisorSpecVersion, + that.supervisorSpecVersion + ); } @Override - public String toString() + public int hashCode() { - return "TaskConfigUpdateRequest{" + - "ioConfig=" + ioConfig + - '}'; + return Objects.hash(ioConfig, supervisorSpecVersion); } + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java index 3515da252c99..faa2167993a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -28,41 +28,51 @@ /** * Request object for updating the configuration of a running {@link SeekableStreamIndexTask}. */ -public class TaskConfigUpdateRequest +public class TaskConfigUpdateRequest { - private final SeekableStreamIndexTaskIOConfig ioConfig; + private final SeekableStreamIndexTaskIOConfig ioConfig; + + private final String supervisorSpecVersion; @JsonCreator public TaskConfigUpdateRequest( - @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig + @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig, + @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion ) { this.ioConfig = ioConfig; + this.supervisorSpecVersion = supervisorSpecVersion; + } + + @JsonProperty + public String getSupervisorSpecVersion() + { + return supervisorSpecVersion; } @JsonProperty - public SeekableStreamIndexTaskIOConfig getIoConfig() + public SeekableStreamIndexTaskIOConfig getIoConfig() { return ioConfig; } @Override - public boolean equals(Object o) + public boolean equals(Object object) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { + if (object == null || getClass() != object.getClass()) { return false; } - TaskConfigUpdateRequest that = (TaskConfigUpdateRequest) o; - return Objects.equals(ioConfig, that.ioConfig); + TaskConfigUpdateRequest that = (TaskConfigUpdateRequest) object; + return Objects.equals(ioConfig, that.ioConfig) && Objects.equals( + supervisorSpecVersion, + that.supervisorSpecVersion + ); } @Override public int hashCode() { - return Objects.hash(ioConfig); + return Objects.hash(ioConfig, supervisorSpecVersion); } @Override @@ -70,6 +80,7 @@ public String toString() { return "TaskConfigUpdateRequest{" + "ioConfig=" + ioConfig + + ", supervisorSpecVersion='" + supervisorSpecVersion + '\'' + '}'; } } 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 16c5899358b3..32aa4eb3c252 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 @@ -73,6 +73,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.TaskConfigResponse; import org.apache.druid.indexing.seekablestream.TaskConfigUpdateRequest; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; @@ -547,7 +548,7 @@ public void handle() } boolean allocationSuccess = changeTaskCount(desiredTaskCount, onSuccessfulScale); - if (allocationSuccess && !spec.usePerpetuallyRunningTasks()) { + if (allocationSuccess && !spec.usePersistentTasks()) { onSuccessfulScale.run(); dynamicTriggerLastRunTime = nowTime; } @@ -599,7 +600,7 @@ private boolean changeTaskCount(int desiredActiveTaskCount, Runnable successfulS ); final Stopwatch scaleActionStopwatch = Stopwatch.createStarted(); - if (spec.usePerpetuallyRunningTasks()) { + if (spec.usePersistentTasks()) { return changeTaskCountForPerpetualTasks(desiredActiveTaskCount, successfulScaleAutoScalerCallback); } else { gracefulShutdownInternal(); @@ -768,13 +769,16 @@ private Map> calculateNewPartitionGroups() * Sends configuration updates to tasks with new partition assignments. * Also handles cleanup of obsolete task groups when scaling down. */ - private boolean sendConfigUpdatesToTasks(Map> newPartitionGroups, Map latestTaskOffsetsOnPause) + private boolean sendConfigUpdatesToTasks( + Map> newPartitionGroups, + Map latestTaskOffsetsOnPause + ) throws InterruptedException, ExecutionException { log.info("Sending configuration updates to the following partition groups %s", newPartitionGroups); List> updateFutures = new ArrayList<>(); Map latestCommittedOffsets = getOffsetsFromMetadataStorage(); - for (Map.Entry entry : activelyReadingTaskGroups.entrySet()) { + for (Entry entry : activelyReadingTaskGroups.entrySet()) { int taskGroupId = entry.getKey(); TaskGroup existingTaskGroup = entry.getValue(); @@ -789,7 +793,8 @@ private boolean sendConfigUpdatesToTasks(Map> newP TaskGroup newTaskGroup = existingTaskGroup.withStartingSequences(newStartingSequences); for (String taskId : existingTaskGroup.taskIds()) { log.info("Updating config for task [%s] with partitions [%s]", taskId, partitionsForThisTask); - updateFutures.add(persistThenUpdateTaskConfig(taskId, newIoConfig)); + TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest<>(newIoConfig, spec.getVersion().get()); + updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); } activelyReadingTaskGroups.put(taskGroupId, newTaskGroup); } @@ -818,43 +823,6 @@ private boolean sendConfigUpdatesToTasks(Map> newP return allSucceeded; } - private ListenableFuture persistThenUpdateTaskConfig( - String taskId, - SeekableStreamIndexTaskIOConfig newIoConfig - ) - { - return Futures.transformAsync( - workerExec.submit(() -> { - Optional existingTaskOpt = taskStorage.getTask(taskId); - if (!existingTaskOpt.isPresent()) { - throw new ISE("Task [%s] not found in storage", taskId); - } - SeekableStreamIndexTask existingTask = - (SeekableStreamIndexTask) existingTaskOpt.get(); - SeekableStreamIndexTask updatedTask = existingTask.withNewIoConfig(newIoConfig); - log.info("Persisting updated config for task [%s] to storage", taskId); - updateTaskIoConfigInQueueOrStorage(updatedTask); - return updatedTask; - }), - (persistedTask) -> { - log.info("Sending config update to running task [%s]", taskId); - TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(newIoConfig); - return taskClient.updateConfigAsync(taskId, updateRequest); - }, - workerExec - ); - } - - private void updateTaskIoConfigInQueueOrStorage(SeekableStreamIndexTask updatedTask) - { - Optional taskQueue = taskMaster.getTaskQueue(); - if (taskQueue.isPresent()) { - taskQueue.get().update(updatedTask); - } else { - taskStorage.updateTask(updatedTask); - } - } - /** * Handles obsolete task groups when scaling down. @@ -926,9 +894,11 @@ protected abstract SeekableStreamIndexTaskIOConfig latestTaskOffsetsOnPause ); + private void changeTaskCountInIOConfig(int desiredActiveTaskCount) { ioConfig.setTaskCount(desiredActiveTaskCount); + spec.setVersion(DateTimes.nowUtc().toString()); try { Optional supervisorManager = taskMaster.getSupervisorManager(); if (supervisorManager.isPresent()) { @@ -1175,7 +1145,7 @@ private boolean isCheckpointSignatureValid( ) { // Earlier checkpoints may no longer have the same partitions. - if (spec.usePerpetuallyRunningTasks()) { + if (spec.usePersistentTasks()) { return true; } @@ -1299,7 +1269,7 @@ public SeekableStreamSupervisor( this.ioConfig = spec.getIoConfig(); this.autoScalerConfig = ioConfig.getAutoScalerConfig(); this.tuningConfig = spec.getTuningConfig(); - this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()); + this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig(spec.usePersistentTasks()); this.supervisorId = spec.getId(); this.exec = Execs.singleThreaded(StringUtils.encodeForFormat(supervisorTag)); this.scheduledExec = Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(supervisorTag) + "-Scheduler-%d"); @@ -1838,7 +1808,7 @@ public Map> getStats() public List getParseErrors() { try { - if (spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()).getMaxParseExceptions() <= 0) { + if (spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePersistentTasks()).getMaxParseExceptions() <= 0) { return ImmutableList.of(); } lastKnownParseErrors = getCurrentParseErrors(); @@ -2004,7 +1974,7 @@ private List getCurrentParseErrors() } // store a limited number of parse exceptions, keeping the most recent ones - int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()).getMaxSavedParseExceptions() * + int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePersistentTasks()).getMaxSavedParseExceptions() * spec.getSpec().getIOConfig().getTaskCount(); parseErrorLimit = Math.min(parseErrorLimit, parseErrorsTreeSet.size()); @@ -2723,7 +2693,7 @@ private void resumeAllActivelyReadingTasks() final String killMsg = "Killing forcefully as task could not be resumed in the first supervisor run after Overlord change."; - for (Map.Entry> entry : tasksToResume.entrySet()) { + for (Entry> entry : tasksToResume.entrySet()) { String taskId = entry.getKey(); ListenableFuture future = entry.getValue(); future.addListener( @@ -3088,6 +3058,11 @@ public boolean isTaskCurrent(int taskGroupId, String taskId, Map a SeekableStreamIndexTask task = (SeekableStreamIndexTask) genericTask; + // If the persistent tasks are present, we should use the version. + if (spec.usePersistentTasks()) { + return isPersistentTaskCurrent(task); + } + // We recompute the sequence name hash for the supervisor's own configuration and compare this to the hash created // by rehashing the task's sequence name using the most up-to-date class definitions of tuning config and // data schema. Recomputing both ensures that forwards-compatible tasks won't be killed (which would occur @@ -3122,6 +3097,23 @@ public boolean isTaskCurrent(int taskGroupId, String taskId, Map a } } + /** + * Verifies whether the running config version of the persistent task matches one in current supervisor spec. + */ + private boolean isPersistentTaskCurrent(SeekableStreamIndexTask task) + { + try { + final String currentVersion = spec.getVersion().get(); + final TaskConfigResponse runningConfig = FutureUtils.get( + taskClient.getTaskConfigAsync(task.getId()), true); + return currentVersion.equals(runningConfig.getSupervisorSpecVersion()); + } + catch (Exception e) { + log.error(e, "Could not fetch running config for task[%s]", task.getId()); + return false; + } + } + @VisibleForTesting public String generateSequenceName( Map startPartitions, @@ -3662,7 +3654,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException stopTasksEarly = false; } // If using perpetually running tasks, we should not stop tasks based on duration - if (spec.usePerpetuallyRunningTasks()) { + if (spec.usePersistentTasks()) { return; } @@ -5036,7 +5028,7 @@ protected void emitLag() final LagStats lagStats = aggregatePartitionLags(partitionLags); Map metricTags = spec.getContextValue(DruidMetrics.TAGS); - for (Map.Entry entry : partitionLags.entrySet()) { + for (Entry entry : partitionLags.entrySet()) { emitter.emit( ServiceMetricEvent.builder() .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) 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 71a5d195c3a8..5f96633ea831 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.DateTimes; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -45,6 +46,7 @@ import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.Optional; public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec { @@ -78,7 +80,8 @@ private static SeekableStreamSupervisorIngestionSpec checkIngestionSchema( protected final DruidMonitorSchedulerConfig monitorSchedulerConfig; private final boolean suspended; protected final SupervisorStateManagerConfig supervisorStateManagerConfig; - protected final boolean usePerpetuallyRunningTasks; + protected final boolean usePersistentTasks; + protected String version; /** * Base constructor for SeekableStreamSupervisors. @@ -99,7 +102,7 @@ public SeekableStreamSupervisorSpec( DruidMonitorSchedulerConfig monitorSchedulerConfig, RowIngestionMetersFactory rowIngestionMetersFactory, SupervisorStateManagerConfig supervisorStateManagerConfig, - @Nullable Boolean usePerpetuallyRunningTasks + @Nullable Boolean usePersistentTasks ) { this.ingestionSchema = checkIngestionSchema(ingestionSchema); @@ -119,7 +122,8 @@ public SeekableStreamSupervisorSpec( this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.suspended = suspended != null ? suspended : false; this.supervisorStateManagerConfig = supervisorStateManagerConfig; - this.usePerpetuallyRunningTasks = Configs.valueOrDefault(usePerpetuallyRunningTasks, false); + this.usePersistentTasks = Configs.valueOrDefault(usePersistentTasks, false); + this.version = DateTimes.nowUtc().toString(); } @JsonProperty @@ -148,9 +152,9 @@ public SeekableStreamSupervisorIOConfig getIoConfig() } @JsonProperty - public boolean usePerpetuallyRunningTasks() + public boolean usePersistentTasks() { - return usePerpetuallyRunningTasks; + return usePersistentTasks; } @Nullable @@ -235,6 +239,16 @@ public boolean isSuspended() return suspended; } + public Optional getVersion() + { + return Optional.of(this.version); + } + + public void setVersion(String version) + { + this.version = version; + } + /** * Default implementation that prevents unsupported evolution of the supervisor spec *
    @@ -265,5 +279,4 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept } protected abstract SeekableStreamSupervisorSpec toggleSuspend(boolean suspend); - } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java index 284538e2a875..7021d23fb50a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorageTest.java @@ -94,23 +94,4 @@ public void testGetTaskInfos() Assert.assertEquals(1, taskInfosComplete.size()); Assert.assertEquals(task1.getId(), taskInfosComplete.get(0).getTask().getId()); } - - @Test - public void testUpdateTask() - { - final NoopTask originalTask = NoopTask.create(); - final TaskStatus runningStatus = TaskStatus.running(originalTask.getId()); - storage.insert(originalTask, runningStatus); - - final NoopTask updatedTask = new NoopTask(originalTask.getId(), null, "testDS", 5000, 0, null); - - storage.updateTask(updatedTask); - - TaskInfo taskInfo = storage.getTaskInfo(originalTask.getId()); - Assert.assertEquals(runningStatus, taskInfo.getStatus()); - Assert.assertEquals(5000L, ((NoopTask) taskInfo.getTask()).getRunTime()); - Assert.assertEquals("testDS", taskInfo.getTask().getDataSource()); - - Assert.assertThrows(IllegalStateException.class, () -> storage.updateTask(NoopTask.create())); - } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java deleted file mode 100644 index 223e501dce01..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/MetadataTaskStorageUpdateTaskTest.java +++ /dev/null @@ -1,117 +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.overlord; - -import com.google.common.base.Optional; -import org.apache.druid.error.DruidException; -import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.metadata.MetadataStorageActionHandler; -import org.apache.druid.metadata.MetadataStorageActionHandlerFactory; -import org.apache.druid.metadata.MetadataStorageConnector; -import org.easymock.EasyMock; -import org.junit.Before; -import org.junit.Test; - -public class MetadataTaskStorageUpdateTaskTest -{ - private MetadataStorageConnector metadataStorageConnector; - private TaskStorageConfig taskStorageConfig; - private MetadataStorageActionHandlerFactory handlerFactory; - private MetadataStorageActionHandler handler; - private MetadataTaskStorage metadataTaskStorage; - - @Before - public void setUp() - { - metadataStorageConnector = EasyMock.createMock(MetadataStorageConnector.class); - taskStorageConfig = EasyMock.createMock(TaskStorageConfig.class); - handlerFactory = EasyMock.createMock(MetadataStorageActionHandlerFactory.class); - handler = EasyMock.createMock(MetadataStorageActionHandler.class); - - EasyMock.expect(handlerFactory.create()).andReturn(handler); - EasyMock.replay(handlerFactory); - - metadataTaskStorage = new MetadataTaskStorage( - metadataStorageConnector, - taskStorageConfig, - handlerFactory - ); - } - - @Test - public void testUpdateTaskSuccess() - { - final NoopTask task = NoopTask.create(); - - EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.of(task)); - handler.update(task.getId(), task); - EasyMock.expectLastCall(); - - EasyMock.replay(handler); - - metadataTaskStorage.updateTask(task); - - EasyMock.verify(handler); - } - - @Test(expected = ISE.class) - public void testUpdateTaskNotFound() - { - final NoopTask task = NoopTask.create(); - - EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.absent()); - - EasyMock.replay(handler); - - metadataTaskStorage.updateTask(task); - } - - @Test(expected = DruidException.class) - public void testUpdateTaskDruidException() - { - final NoopTask task = NoopTask.create(); - final DruidException druidException = DruidException.defensive("Test DruidException"); - - EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.of(task)); - handler.update(task.getId(), task); - EasyMock.expectLastCall().andThrow(druidException); - - EasyMock.replay(handler); - - metadataTaskStorage.updateTask(task); - } - - @Test(expected = RuntimeException.class) - public void testUpdateTaskGenericException() - { - final NoopTask task = NoopTask.create(); - final Exception genericException = new Exception("Test generic exception"); - - EasyMock.expect(handler.getEntry(task.getId())).andReturn(Optional.of(task)); - handler.update(task.getId(), task); - EasyMock.expectLastCall().andThrow(genericException); - - EasyMock.replay(handler); - - metadataTaskStorage.updateTask(task); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index c4e08d0054e8..662966067ac1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -74,7 +74,6 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -691,35 +690,6 @@ public TaskStatus runTask(TaskToolbox toolbox) Assert.assertEquals(failedStatus, getTaskStorage().getStatus(task.getId()).get()); } - @Test - public void test_update_successfullyUpdates() - { - final TestTask originalTask = new TestTask("update-test-task", Intervals.of("2021-01-01/P1D")); - taskQueue.add(originalTask); - - final TestTask updatedTask = new TestTask("update-test-task", "updatedDatasource", Intervals.of("2021-01-01/P1D"), - ImmutableMap.of("testKey", "testValue")); - - taskQueue.update(updatedTask); - - final Optional updatedInfoOpt = taskQueue.getActiveTaskInfo(updatedTask.getId()); - Assert.assertTrue(updatedInfoOpt.isPresent()); - Assert.assertEquals("updatedDatasource", updatedInfoOpt.get().getTask().getDataSource()); - Assert.assertEquals("testValue", updatedInfoOpt.get().getTask().getContextValue("testKey")); - - final Optional taskInStorage = getTaskStorage().getTask(updatedTask.getId()); - Assert.assertTrue(taskInStorage.isPresent()); - Assert.assertEquals("updatedDatasource", taskInStorage.get().getDataSource()); - } - - @Test(expected = ISE.class) - public void test_update_nonExistingTask() - { - taskQueue.update(new TestTask("non-existing-task", Intervals.of("2021-01-01/P1D"))); - } - - - private HttpRemoteTaskRunner createHttpRemoteTaskRunner() { final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java index e95542a1a4d8..48c53921c733 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java @@ -643,7 +643,7 @@ public void test_pauseAndCheckpointAsync_notAvailable() throws Exception @Test public void test_updateConfigAsync_success() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/config") @@ -660,7 +660,7 @@ public void test_updateConfigAsync_success() throws Exception @Test public void test_updateConfigAsync_httpError() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/config") @@ -680,7 +680,7 @@ public void test_updateConfigAsync_httpError() throws Exception @Test public void test_updateConfigAsync_notAvailable() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/config") @@ -695,7 +695,7 @@ public void test_updateConfigAsync_notAvailable() throws Exception @Test public void test_updateConfigAsync_closed() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/config") diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index d9671e20239e..25d106aebca7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -375,7 +375,7 @@ public TestSeekableStreamIndexTask( SeekableStreamIndexTaskIOConfig ioConfig ) { - super(id, null, null, dataSchema, tuningConfig, ioConfig, null, null, null); + super(id, null, null, dataSchema, tuningConfig, ioConfig, null, null, null, "v1"); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index d52ae0c9056b..658796adf301 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -755,12 +755,14 @@ public void testSeekableStreamSupervisorSpecWithScaleOut() throws InterruptedExc EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig(1, true)).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -817,7 +819,7 @@ public void testSeekableStreamSupervisorSpecWithScaleOutAlreadyAtMax() throws In EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -877,7 +879,7 @@ public void test_dynamicAllocationNotice_skipsScalingAndEmitsReason_ifTasksArePu EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -944,7 +946,7 @@ public void testSeekableStreamSupervisorSpecWithNoScalingOnIdleSupervisor() thro EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -990,12 +992,14 @@ public void testSeekableStreamSupervisorSpecWithScaleOutSmallPartitionNumber() t EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig(1, true)).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1042,7 +1046,9 @@ public void testSeekableStreamSupervisorSpecWithScaleIn() throws InterruptedExce EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1093,7 +1099,9 @@ public void testSeekableStreamSupervisorSpecWithScaleInThresholdGreaterThanParti EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1150,7 +1158,7 @@ public void testSeekableStreamSupervisorSpecWithScaleInAlreadyAtMin() throws Int EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1231,7 +1239,7 @@ public void testSeekableStreamSupervisorSpecWithScaleDisable() throws Interrupte EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(this.seekableStreamSupervisorIOConfig).anyTimes(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java index b853fb03aadb..bad2e95747db 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java @@ -31,13 +31,13 @@ public void testTaskConfigUpdateRequest() SeekableStreamIndexTaskIOConfig mockIoConfig = EasyMock.createMock(SeekableStreamIndexTaskIOConfig.class); EasyMock.replay(mockIoConfig); - TaskConfigUpdateRequest request = new TaskConfigUpdateRequest(mockIoConfig); + TaskConfigUpdateRequest request = new TaskConfigUpdateRequest(mockIoConfig, null); Assert.assertEquals(mockIoConfig, request.getIoConfig()); - TaskConfigUpdateRequest nullRequest = new TaskConfigUpdateRequest(null); + TaskConfigUpdateRequest nullRequest = new TaskConfigUpdateRequest(null, null); Assert.assertNull(nullRequest.getIoConfig()); - TaskConfigUpdateRequest request2 = new TaskConfigUpdateRequest(mockIoConfig); + TaskConfigUpdateRequest request2 = new TaskConfigUpdateRequest(mockIoConfig, null); Assert.assertEquals(request, request2); Assert.assertEquals(request.hashCode(), request2.hashCode()); @@ -47,6 +47,7 @@ public void testTaskConfigUpdateRequest() String toString = request.toString(); Assert.assertTrue(toString.contains("TaskConfigUpdateRequest")); Assert.assertTrue(toString.contains("ioConfig")); + Assert.assertTrue(toString.contains("supervisorSpecVersion")); EasyMock.verify(mockIoConfig); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java index 47f80d17e18d..363f4c0394bb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java @@ -112,8 +112,7 @@ public void testIsMoreToReadBeforeReadingRecord_nullCurrentSequenceNumber_exclus TestSequenceNumber current = new TestSequenceNumber(null, false); TestSequenceNumber end = new TestSequenceNumber(10L, false); - Assert.assertFalse("Should return false when current sequence number is null", - current.isMoreToReadBeforeReadingRecord(end, true)); + Assert.assertThrows(NullPointerException.class, () -> current.isMoreToReadBeforeReadingRecord(end, true)); } @Test @@ -122,8 +121,7 @@ public void testIsMoreToReadBeforeReadingRecord_nullCurrentSequenceNumber_inclus TestSequenceNumber current = new TestSequenceNumber(null, false); TestSequenceNumber end = new TestSequenceNumber(10L, false); - Assert.assertFalse("Should return false when current sequence number is null", - current.isMoreToReadBeforeReadingRecord(end, false)); + Assert.assertThrows(NullPointerException.class, () -> current.isMoreToReadBeforeReadingRecord(end, false)); } @Test @@ -144,10 +142,8 @@ public void testIsMoreToReadBeforeReadingRecord_exceptionHandling() TestExceptionSequenceNumber current = new TestExceptionSequenceNumber(5L, false); TestExceptionSequenceNumber end = new TestExceptionSequenceNumber(10L, false); - Assert.assertFalse("Should return false when comparison throws exception", - current.isMoreToReadBeforeReadingRecord(end, true)); - Assert.assertFalse("Should return false when comparison throws exception", - current.isMoreToReadBeforeReadingRecord(end, false)); + Assert.assertThrows(RuntimeException.class, () -> current.isMoreToReadBeforeReadingRecord(end, true)); + Assert.assertThrows(RuntimeException.class, () -> current.isMoreToReadBeforeReadingRecord(end, false)); } @Test 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 27fd7e8aa307..275c0fbd0d6e 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 @@ -207,7 +207,7 @@ public void setupTest() public void testRunning() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -250,7 +250,7 @@ public void testRunningStreamGetSequenceNumberReturnsNull() EasyMock.expect(recordSupplier.getAssignment()).andReturn(ImmutableSet.of(SHARD0_PARTITION)).anyTimes(); EasyMock.expect(recordSupplier.getLatestSequenceNumber(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -300,7 +300,7 @@ public void testRunningStreamGetSequenceNumberReturnsNull() public void testAddDiscoveredTaskToPendingCompletionTaskGroups() throws Exception { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -405,7 +405,7 @@ public void testAddDiscoveredTaskToPendingCompletionTaskGroups() throws Exceptio public void testAddDiscoveredTaskToPendingCompletionMultipleTaskGroups() throws Exception { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -477,7 +477,7 @@ public void testAddDiscoveredTaskToPendingCompletionMultipleTaskGroups() throws public void testConnectingToStreamFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)) .andThrow(new StreamException(new IllegalStateException(EXCEPTION_MSG))) .anyTimes(); @@ -533,7 +533,7 @@ public void testConnectingToStreamFail() public void testConnectingToStreamFailRecoveryFailRecovery() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)) .andThrow(new StreamException(new IllegalStateException())) .times(3); @@ -608,7 +608,7 @@ public void testConnectingToStreamFailRecoveryFailRecovery() public void testDiscoveringInitialTasksFailRecoveryFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).times(6); @@ -718,7 +718,7 @@ public Duration getEmissionDuration() } }).anyTimes(); EasyMock.expect(spec.getType()).andReturn("test").anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); @@ -792,7 +792,7 @@ public void testIdleOnStartUpAndTurnsToRunningAfterLagUpdates() EasyMock.reset(spec); EasyMock.expect(spec.getId()).andReturn(SUPERVISOR_ID).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getContextValue("tags")).andReturn("").anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(new SeekableStreamSupervisorIOConfig( @@ -868,7 +868,7 @@ public Duration getEmissionDuration() public void testCreatingTasksFailRecoveryFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); @@ -947,7 +947,7 @@ public void testCreatingTasksFailRecoveryFail() public void testSuspended() { EasyMock.expect(spec.isSuspended()).andReturn(true).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -987,7 +987,7 @@ public void testSuspended() public void testStopping() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -1032,7 +1032,7 @@ public void testStopGracefully() throws Exception EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); taskRunner.unregisterListener("testSupervisorId"); indexTaskClient.close(); @@ -1052,7 +1052,7 @@ public void testStopGracefully() throws Exception public void testStoppingGracefully() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -1125,7 +1125,7 @@ public void testCheckpointForActiveTaskGroup() throws InterruptedException, Json EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() { @Override public Duration getEmissionDuration() @@ -1338,7 +1338,7 @@ public void testEarlyStoppingOfTaskGroupBasedOnStopTaskCount() throws Interrupte EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() @@ -1570,7 +1570,7 @@ public void testSupervisorStopTaskGroupEarly() throws JsonProcessingException, I EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andStubReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andStubReturn(false); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() { @Override @@ -1874,7 +1874,7 @@ public void testEmitNoLagWhenSuspended() throws Exception public void testGetStats() { EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexTaskClient.getMovingAveragesAsync("task1")) .andReturn(Futures.immediateFuture(ImmutableMap.of("prop1", "val1"))) .times(1); @@ -1922,7 +1922,7 @@ public void testGetStats() public void testSupervisorResetAllWithCheckpoints() throws InterruptedException { EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(SUPERVISOR_ID)).andReturn( true ); @@ -1976,7 +1976,7 @@ public void testSupervisorResetOneTaskSpecificOffsetsWithCheckpoints() throws In ) ) ); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata(SUPERVISOR_ID, new TestSeekableStreamDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>( STREAM, @@ -2023,7 +2023,7 @@ public void testSupervisorResetOneTaskSpecificOffsetsWithCheckpoints() throws In public void testRegisterNewVersionOfPendingSegment() { EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); Capture captured0 = Capture.newInstance(CaptureType.FIRST); Capture captured1 = Capture.newInstance(CaptureType.FIRST); EasyMock.expect( @@ -2105,7 +2105,7 @@ public void testSupervisorResetSpecificOffsetsTasksWithCheckpoints() throws Inte final ImmutableMap checkpointOffsets = ImmutableMap.of("0", "5", "1", "6", "2", "100"); final ImmutableMap resetOffsets = ImmutableMap.of("0", "10", "1", "8"); final ImmutableMap expectedOffsets = ImmutableMap.of("0", "10", "1", "8", "2", "100"); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( @@ -2185,7 +2185,7 @@ public void testSupervisorResetOffsetsWithNoCheckpoints() throws InterruptedExce final ImmutableMap expectedOffsets = ImmutableMap.copyOf(resetOffsets); EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn(null); EasyMock.expect(indexerMetadataStorageCoordinator.insertDataSourceMetadata(SUPERVISOR_ID, new TestSeekableStreamDataSourceMetadata( @@ -2259,7 +2259,7 @@ public void testSupervisorResetWithNoPartitions() throws IOException, Interrupte final ImmutableMap expectedOffsets = ImmutableMap.of("0", "5", "1", "6"); EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( new TestSeekableStreamDataSourceMetadata( @@ -2325,7 +2325,7 @@ public void testSupervisorResetWithNewPartition() throws IOException, Interrupte final ImmutableMap expectedOffsets = ImmutableMap.of("0", "5", "1", "6", "2", "20"); EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( new TestSeekableStreamDataSourceMetadata( @@ -2389,7 +2389,7 @@ public void testSupervisorResetWithNewPartition() throws IOException, Interrupte public void testSupervisorNoResetDataSourceMetadata() { EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2429,7 +2429,7 @@ public void testSupervisorNoResetDataSourceMetadata() public void testSupervisorResetWithInvalidStartSequenceMetadata() { EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2480,7 +2480,7 @@ public void testSupervisorResetWithInvalidStartSequenceMetadata() public void testSupervisorResetInvalidStream() { EasyMock.expect(spec.isSuspended()).andReturn(false); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2569,7 +2569,7 @@ private void validateSupervisorStateAfterResetOffsets( public void testScheduleReporting() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); DruidMonitorSchedulerConfig config = new DruidMonitorSchedulerConfig(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(config).times(2); ScheduledExecutorService executorService = EasyMock.createMock(ScheduledExecutorService.class); @@ -2672,7 +2672,7 @@ private void expectEmitterSupervisor(boolean suspended) spec = createMock(SeekableStreamSupervisorSpec.class); EasyMock.expect(spec.getId()).andReturn(SUPERVISOR_ID).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); - EasyMock.expect(spec.usePerpetuallyRunningTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(new SeekableStreamSupervisorIOConfig( "stream", @@ -2980,7 +2980,8 @@ public TestSeekableStreamIndexTask( ioConfig, context, groupId, - null + null, + "v1" ); this.streamingTaskRunner = streamingTaskRunner; } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java index 9ff217d5404a..f1237be74745 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java @@ -28,6 +28,7 @@ import javax.annotation.Nonnull; import java.util.List; +import java.util.Optional; import java.util.Set; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @@ -113,4 +114,9 @@ default void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcep { // The default implementation does not do any validation checks. } + + default Optional getVersion() + { + return Optional.empty(); + } } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java index 2882286de65e..192a5be72bc8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java @@ -89,6 +89,7 @@ public void insert(final String id, final SupervisorSpec spec) { dbi.withHandle( handle -> { + String created_date = spec.getVersion().orElse(DateTimes.nowUtc().toString()); handle.createStatement( StringUtils.format( "INSERT INTO %s (spec_id, created_date, payload) VALUES (:spec_id, :created_date, :payload)", @@ -96,7 +97,7 @@ public void insert(final String id, final SupervisorSpec spec) ) ) .bind("spec_id", id) - .bind("created_date", DateTimes.nowUtc().toString()) + .bind("created_date", created_date) .bind("payload", jsonMapper.writeValueAsBytes(spec)) .execute(); 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 a1428d535888..88e4f64afa58 100644 --- a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java +++ b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java @@ -266,7 +266,7 @@ private TestStreamingTask( ) { - super(id, supervisorId, taskResource, dataSchema, tuningConfig, ioConfig, context, groupId, null); + super(id, supervisorId, taskResource, dataSchema, tuningConfig, ioConfig, context, groupId, null, "v1"); } @Override From 8ce15d2cf8b4eec560a4c9e8bf771646f3761791 Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Thu, 25 Sep 2025 09:38:54 +0530 Subject: [PATCH 44/46] Rename a few tests --- .../kafka/supervisor/KafkaSupervisor.java | 35 +---- .../kafka/supervisor/KafkaSupervisorTest.java | 123 +++++++++++------- 2 files changed, 78 insertions(+), 80 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 32f6bc09b99e..0f54232221bd 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 @@ -141,42 +141,11 @@ protected RecordSupplier setupReco protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) { Integer taskCount = spec.getSpec().getIOConfig().getTaskCount(); - - if (spec.usePersistentTasks()) { - int taskGroupId = getRangeBasedTaskGroupId(partitionId, taskCount); - log.debug("Range-based assignment for partition [%s]: taskGroupId [%d] when taskCount is [%d]", partitionId, taskGroupId, taskCount); - return taskGroupId; - } else { - if (partitionId.isMultiTopicPartition()) { - return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; - } else { - return partitionId.partition() % taskCount; - } - } - } - - @VisibleForTesting - public int getRangeBasedTaskGroupIdForPartition(KafkaTopicPartition partitionId, Integer taskCount, int totalPartitions) - { - int minPartitionsPerTaskGroup = totalPartitions / taskCount; - if (partitionId.isMultiTopicPartition()) { return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; + } else { + return partitionId.partition() % taskCount; } - int taskGroup = (partitionId.partition() / minPartitionsPerTaskGroup); - if (taskGroup >= taskCount) { - taskGroup--; - } - return taskGroup; - } - - /** - * Assigns partitions to task groups using range-based sequential assignment. - * This ensures that adjacent partitions are assigned to the same task group - */ - private int getRangeBasedTaskGroupId(KafkaTopicPartition partitionId, Integer taskCount) - { - return getRangeBasedTaskGroupIdForPartition(partitionId, taskCount, partitionIds.size()); } @Override 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 9449d39d41ae..e3816763bb46 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 @@ -69,6 +69,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.TaskConfigResponse; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; @@ -245,6 +246,9 @@ public void setupTest() EasyMock.expect(taskClient.pauseAndCheckpointAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(dummyOffsets)) .anyTimes(); + EasyMock.expect(taskClient.getTaskConfigAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(new TaskConfigResponse<>(null, "v1"))) + .anyTimes(); EasyMock.expect(taskClient.updateConfigAsync( EasyMock.anyString(), EasyMock.anyObject() @@ -367,7 +371,14 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, new SupervisorStateManagerConfig() - ); + ) + { + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of("v1"); + } + }; supervisor = new TestableKafkaSupervisor( taskStorage, @@ -474,7 +485,7 @@ public SeekableStreamIndexTaskClient build( } @Test - public void test_noInitialState_withPerpetuallyRunningTasks() throws Exception + public void test_noInitialState_withPersistentTasks() throws Exception { final int taskCountMax = 2; final int replicas = 1; @@ -566,7 +577,14 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, new SupervisorStateManagerConfig() - ); + ) + { + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of("v1"); + } + }; supervisor = new TestableKafkaSupervisor( taskStorage, @@ -580,8 +598,6 @@ public SeekableStreamIndexTaskClient build( SupervisorTaskAutoScaler autoscaler = testableSupervisorSpec.createAutoscaler(supervisor); - - final KafkaSupervisorTuningConfig tuningConfig = supervisor.getTuningConfig(); addSomeEvents(1); Capture captured = Capture.newInstance(); @@ -615,9 +631,10 @@ public SeekableStreamIndexTaskClient build( EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)) .andReturn(createdTasks).anyTimes(); + EasyMock.expect(taskClient.getTaskConfigAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(new TaskConfigResponse<>(null, "v1"))) + .anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); - taskQueue.update(EasyMock.anyObject()); - EasyMock.expectLastCall().anyTimes(); EasyMock.expect(taskStorage.getStatus(createdTask.getId())) .andReturn(Optional.of(TaskStatus.running(createdTask.getId()))).anyTimes(); EasyMock.expect(taskStorage.getTask(createdTask.getId())) @@ -5260,38 +5277,6 @@ public void test_doesTaskMatchSupervisor() EasyMock.replay(differentTaskType); } - @Test - public void test_getTaskGroupIdForPartition() throws Exception - { - int taskCount = 3; - supervisor = getTestableSupervisor( - "test-supervisor", - 1, - taskCount, - true, - false, - "PT1H", - null, - null, - false, - kafkaHost, - null, - true - ); - - int totalPartitions = 10; - Assert.assertEquals(0, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 0), taskCount, totalPartitions)); - Assert.assertEquals(0, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 1), taskCount, totalPartitions)); - Assert.assertEquals(0, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 2), taskCount, totalPartitions)); - Assert.assertEquals(1, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 3), taskCount, totalPartitions)); - Assert.assertEquals(1, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 4), taskCount, totalPartitions)); - Assert.assertEquals(1, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 5), taskCount, totalPartitions)); - Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 6), taskCount, totalPartitions)); - Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 7), taskCount, totalPartitions)); - Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 8), taskCount, totalPartitions)); - Assert.assertEquals(2, supervisor.getRangeBasedTaskGroupIdForPartition(new KafkaTopicPartition(false, topic, 9), taskCount, totalPartitions)); - } - private void addSomeEvents(int numEventsPerPartition) throws Exception { // create topic manually @@ -5478,13 +5463,13 @@ private TestableKafkaSupervisor getTestableSupervisor( replicas, taskCount, useEarliestOffset, - false, + resetOffsetAutomatically, duration, lateMessageRejectionPeriod, earlyMessageRejectionPeriod, suspended, kafkaHost, - null, + idleConfig, null ); } @@ -5501,7 +5486,7 @@ private TestableKafkaSupervisor getTestableSupervisor( boolean suspended, String kafkaHost, IdleConfig idleConfig, - Boolean usePerpetuallyRunningTasks + Boolean usePersistentTasks ) { final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); @@ -5569,7 +5554,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, - usePerpetuallyRunningTasks, + usePersistentTasks, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -5579,7 +5564,22 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, supervisorConfig - ), + ) + { + private String version = "v1"; + + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of(version); + } + + @Override + public void setVersion(String newVersion) + { + this.version = newVersion; + } + }, rowIngestionMetersFactory ); } @@ -5671,7 +5671,22 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, supervisorConfig - ), + ) + { + private String version = "v1"; + + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of(version); + } + + @Override + public void setVersion(String newVersion) + { + this.version = newVersion; + } + }, rowIngestionMetersFactory, isTaskCurrentReturn ); @@ -5764,7 +5779,22 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, supervisorConfig - ), + ) + { + private String version = "v1"; + + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of(version); + } + + @Override + public void setVersion(String newVersion) + { + this.version = newVersion; + } + }, rowIngestionMetersFactory ); } @@ -6029,7 +6059,6 @@ public String generateSequenceName( final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next()); return StringUtils.format("sequenceName-%d", groupId); } - private SeekableStreamSupervisorStateManager getStateManager() { return stateManager; From f0997ba98898e6a41bb7c20e51e4475b48389d3f Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Fri, 26 Sep 2025 13:16:56 +0530 Subject: [PATCH 45/46] Add new embedded tests, fix no data pushed bug, revert some refactors --- .../indexing/KafkaClusterMetricsTest.java | 164 +---------- .../indexing/KafkaTaskScalingTest.java | 265 ++++++++++++++++++ .../supervisor/RabbitStreamSupervisor.java | 10 +- .../kafka/supervisor/KafkaSupervisor.java | 28 +- .../KafkaSupervisorSpecBuilder.java | 8 +- .../SeekableStreamIndexTaskRunner.java | 24 +- .../TaskConfigUpdateRequest.java | 13 +- .../supervisor/SeekableStreamSupervisor.java | 4 +- ...bleStreamIndexTaskClientAsyncImplTest.java | 8 +- .../TaskConfigUpdateRequestTest.java | 6 +- 10 files changed, 331 insertions(+), 199 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index fdf144dc58d7..155c8a6beb4b 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -27,9 +27,7 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; import org.apache.druid.indexing.kafka.simulate.KafkaResource; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; -import org.apache.druid.indexing.kafka.supervisor.LagBasedAutoScalerConfigBuilder; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.rpc.indexing.OverlordClient; @@ -147,8 +145,7 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( supervisorId, taskCount, - maxRowsPerSegment, - null + maxRowsPerSegment ); Assertions.assertEquals( @@ -181,74 +178,6 @@ public void test_ingest10kRows_ofSelfClusterMetrics_andVerifyValues() cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); } - @Test - @Timeout(60) - public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() - { - final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 20; - - final int taskCount = 1; - - // Submit and start a supervisor - final String supervisorId = dataSource + "_supe"; - AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() - .withLagCollectionIntervalMillis(100) - .withLagCollectionRangeMillis(100) - .withEnableTaskAutoScaler(true) - .withScaleActionPeriodMillis(5000) - .withScaleActionStartDelayMillis(10000) - .withScaleOutThreshold(0) - .withScaleInThreshold(10000) - .withTriggerScaleOutFractionThreshold(0.001) - .withTriggerScaleInFractionThreshold(0.1) - .withTaskCountMax(3) - .withTaskCountMin(taskCount) - .withScaleOutStep(1) - .withScaleInStep(0) - .withMinTriggerScaleActionFrequencyMillis(5000) - .withStopTaskCountRatio(1.0) - .build(); - - final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( - supervisorId, - taskCount, - maxRowsPerSegment, - autoScalerConfig, - true - ); - - Assertions.assertEquals( - supervisorId, - cluster.callApi().postSupervisor(kafkaSupervisorSpec) - ); - overlord.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("task/autoScaler/scaleActionTime"), - agg -> agg.hasSumAtLeast(2) - ); - - indexer.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("ingest/handoff/count") - .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)), - agg -> agg.hasSumAtLeast(expectedSegmentsHandedOff) - ); - - final int numSegments = Integer.parseInt( - cluster.runSql("SELECT COUNT(*) FROM sys.segments WHERE datasource = '%s'", dataSource) - ); - Assertions.assertTrue(numSegments >= expectedSegmentsHandedOff); - - final int numRows = Integer.parseInt( - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); - Assertions.assertTrue(numRows >= expectedSegmentsHandedOff * maxRowsPerSegment); - - verifyIngestedMetricCountMatchesEmittedCount("jvm/pool/committed", coordinator); - verifyIngestedMetricCountMatchesEmittedCount("coordinator/time", coordinator); - - cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); - } - @Test @Timeout(120) public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkipKillOfUnusedSegments() @@ -263,8 +192,7 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( supervisorId, taskCount, - maxRowsPerSegment, - null + maxRowsPerSegment ); cluster.callApi().postSupervisor(kafkaSupervisorSpec); @@ -367,75 +295,6 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); } - @Test - @Timeout(60) - public void test_ingest20kRows_ofSelfClusterMetricsWithScaleIns_andVerifyValues() - { - final int maxRowsPerSegment = 1000; - final int expectedSegmentsHandedOff = 20; - - final int initialTaskCount = 3; - - // Submit and start a supervisor with scale-in configuration - final String supervisorId = dataSource + "_supe"; - AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() - .withLagCollectionIntervalMillis(500) - .withLagCollectionRangeMillis(1000) - .withEnableTaskAutoScaler(true) - .withScaleActionPeriodMillis(10000) - .withScaleActionStartDelayMillis(5000) - .withScaleOutThreshold(10000) - .withScaleInThreshold(1) - .withTriggerScaleOutFractionThreshold(0.9) - .withTriggerScaleInFractionThreshold(0.001) - .withTaskCountMax(initialTaskCount) - .withTaskCountStart(initialTaskCount) - .withScaleOutStep(0) - .withScaleInStep(1) - .withMinTriggerScaleActionFrequencyMillis(10000) - .withStopTaskCountRatio(1.0) - .build(); - - final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( - supervisorId, - initialTaskCount, - maxRowsPerSegment, - autoScalerConfig, - true - ); - - Assertions.assertEquals( - supervisorId, - cluster.callApi().postSupervisor(kafkaSupervisorSpec) - ); - - overlord.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("task/autoScaler/scaleActionTime"), - agg -> agg.hasSumAtLeast(2) - ); - - indexer.latchableEmitter().waitForEventAggregate( - event -> event.hasMetricName("ingest/handoff/count") - .hasDimension(DruidMetrics.DATASOURCE, List.of(dataSource)), - agg -> agg.hasSumAtLeast(expectedSegmentsHandedOff) - ); - - final int numSegments = Integer.parseInt( - cluster.runSql("SELECT COUNT(*) FROM sys.segments WHERE datasource = '%s'", dataSource) - ); - Assertions.assertTrue(numSegments >= expectedSegmentsHandedOff); - - final int numRows = Integer.parseInt( - cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) - ); - Assertions.assertTrue(numRows >= expectedSegmentsHandedOff * maxRowsPerSegment); - - verifyIngestedMetricCountMatchesEmittedCount("jvm/pool/committed", coordinator); - verifyIngestedMetricCountMatchesEmittedCount("coordinator/time", coordinator); - - cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); - } - @Test @Timeout(120) public void test_ingestClusterMetrics_compactionSkipsLockedIntervals() @@ -450,8 +309,7 @@ public void test_ingestClusterMetrics_compactionSkipsLockedIntervals() final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisor( supervisorId, taskCount, - maxRowsPerSegment, - null + maxRowsPerSegment ); cluster.callApi().postSupervisor(kafkaSupervisorSpec); @@ -530,19 +388,7 @@ private void verifyIngestedMetricCountMatchesEmittedCount(String metricName, Emb private KafkaSupervisorSpec createKafkaSupervisor( String supervisorId, int taskCount, - int maxRowsPerSegment, - AutoScalerConfig autoScalerConfig - ) - { - return createKafkaSupervisor(supervisorId, taskCount, maxRowsPerSegment, autoScalerConfig, false); - } - - private KafkaSupervisorSpec createKafkaSupervisor( - String supervisorId, - int taskCount, - int maxRowsPerSegment, - AutoScalerConfig autoScalerConfig, - boolean usePerpetuallyRunningTasks + int maxRowsPerSegment ) { return MoreResources.Supervisor.KAFKA_JSON @@ -556,10 +402,8 @@ private KafkaSupervisorSpec createKafkaSupervisor( ioConfig -> ioConfig .withConsumerProperties(kafkaServer.consumerProperties()) .withTaskCount(taskCount) - .withAutoScalerConfig(autoScalerConfig) ) .withId(supervisorId) - .withUsePerpetuallyRunningTasks(usePerpetuallyRunningTasks) .build(dataSource, TOPIC); } } diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java new file mode 100644 index 000000000000..0e778a631cc2 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java @@ -0,0 +1,265 @@ +/* + * 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.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.LagBasedAutoScalerConfigBuilder; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +/** + * Embedded test to verify task scaling behaviour of {@code KafkaSupervisor} ingesting from a custom kafka topic. + */ +@SuppressWarnings("resource") +public class KafkaTaskScalingTest extends EmbeddedClusterTestBase +{ + private static final String TOPIC = EmbeddedClusterApis.createTestDatasourceName(); + + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + + private KafkaResource kafkaServer; + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + createTopicWithPartitions(TOPIC, 10); + } + + @Override + public void stop() + { + deleteTopic(TOPIC); + super.stop(); + } + }; + + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.server.http.numThreads", "50") + .addProperty("druid.manager.segments.pollDuration", "PT0.1s"); + cluster.addExtension(KafkaIndexTaskModule.class) + .addResource(kafkaServer) + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(historical) + .addServer(new EmbeddedRouter()) + .useLatchableEmitter(); + + return cluster; + } + + @Test + @Timeout(10) + public void test_supervisorTasksFinish_withNoDataAndShortTaskDuration() + { + final int taskCount = 3; + + final String supervisorId = dataSource + "_short_tasks"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpec(supervisorId, taskCount, null, false); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + Assertions.fail("Test interrupted"); + } + + final String successTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'SUCCESS'", + dataSource + ); + final int successfulTasks = Integer.parseInt(successTaskCountResult); + + Assertions.assertEquals( + taskCount, + successfulTasks, + String.format("Expected all %d tasks to succeed, but only %d succeeded", taskCount, successfulTasks) + ); + + cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); + } + + @Test + @Timeout(20) + public void test_supervisorTasksDontFinish_withPersistentTasks() + { + final int taskCount = 3; + + final String supervisorId = dataSource + "_persistent_short_tasks"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpec(supervisorId, taskCount, null, true); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + try { + Thread.sleep(10000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + Assertions.fail("Test interrupted"); + } + + final String runningTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", + dataSource + ); + final int runningTasks = Integer.parseInt(runningTaskCountResult); + + Assertions.assertEquals( + taskCount, + runningTasks, + String.format("Expected all %d tasks to be running, but only %d were found running", taskCount, runningTasks) + ); + + cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); + } + + @Test + @Timeout(20) + public void test_supervisorTasksScalesIn_withPersistentTasksAndAutoScaler() + { + final int initialTaskCount = 3; + final int taskCountMin = 1; + + AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() + .withLagCollectionIntervalMillis(500) + .withLagCollectionRangeMillis(1000) + .withEnableTaskAutoScaler(true) + .withScaleActionPeriodMillis(5000) + .withScaleActionStartDelayMillis(5000) + .withScaleOutThreshold(10000) + .withScaleInThreshold(1) + .withTaskCountMin(taskCountMin) + .withTriggerScaleOutFractionThreshold(0.9) + .withTriggerScaleInFractionThreshold(0.001) + .withTaskCountMax(initialTaskCount) + .withTaskCountStart(initialTaskCount) + .withScaleOutStep(0) + .withScaleInStep(1) + .withMinTriggerScaleActionFrequencyMillis(1000) + .withStopTaskCountRatio(1.0) + .build(); + + String supervisorId = dataSource + "_persistent_autoscale_tasks"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpec( + supervisorId, + initialTaskCount, + autoScalerConfig, + true + ); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("task/autoScaler/scaleActionTime"), + agg -> agg.hasCountAtLeast(2) + ); + + try { + Thread.sleep(2000); // Wait for a few seconds for the tasks to scale in + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + Assertions.fail("Test interrupted"); + } + + final String runningTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", + dataSource + ); + + final int runningTasks = Integer.parseInt(runningTaskCountResult); + + Assertions.assertEquals( + taskCountMin, + runningTasks, + String.format("Expected all %d tasks to be running, but only %d were found running", taskCountMin, runningTasks) + ); + } + + @Test + @Timeout(20) + public void tesT_supervisorTasksScalesOutAndScalesIn_withPersistentTasksAndAutoScaler() + { + + } + + private KafkaSupervisorSpec createSupervisorSpec( + String supervisorId, + int taskCount, + AutoScalerConfig autoScalerConfig, + boolean usePersistentTasks + ) + { + return MoreResources.Supervisor.KAFKA_JSON + .get() + .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) + .withTuningConfig(tuningConfig -> tuningConfig + .withMaxRowsPerSegment(1000) + .withWorkerThreads(10)) + .withIoConfig( + ioConfig -> ioConfig + .withConsumerProperties(kafkaServer.consumerProperties()) + .withTaskCount(taskCount) + .withTaskDuration(Period.millis(500)) + .withAutoScalerConfig(autoScalerConfig) + ) + .withId(supervisorId) + .withUsePersistentTasks(usePersistentTasks) + .build(dataSource, TOPIC); + } +} 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 60446d7aa7a7..b4a5a14b8440 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 @@ -121,9 +121,9 @@ protected RecordSupplier setupRecordSupplier() RabbitStreamIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig(); return new RabbitStreamRecordSupplier( - spec.getSpec().getIOConfig().getConsumerProperties(), + spec.getIoConfig().getConsumerProperties(), sortingMapper, - spec.getSpec().getIOConfig().getUri(), + spec.getIoConfig().getUri(), taskTuningConfig.getRecordBufferSizeOrDefault(Runtime.getRuntime().maxMemory()), taskTuningConfig.getRecordBufferOfferTimeout(), taskTuningConfig.getMaxRecordsPerPollOrDefault() @@ -133,7 +133,7 @@ protected RecordSupplier setupRecordSupplier() @Override protected int getTaskGroupIdForPartition(String partitionId) { - return partitionId.hashCode() % spec.getSpec().getIOConfig().getTaskCount(); + return partitionId.hashCode() % spec.getIoConfig().getTaskCount(); } @Override @@ -158,7 +158,7 @@ protected SeekableStreamSupervisorReportPayload createReportPayloa int numPartitions, boolean includeOffsets) { - RabbitStreamSupervisorIOConfig ioConfig = spec.getSpec().getIOConfig(); + RabbitStreamSupervisorIOConfig ioConfig = spec.getIoConfig(); Map partitionLag = getRecordLagPerPartitionInLatestSequences(getHighestCurrentOffsets()); return new RabbitStreamSupervisorReportPayload( spec.getId(), @@ -423,7 +423,7 @@ protected String baseTaskName() @VisibleForTesting public RabbitStreamSupervisorIOConfig getIoConfig() { - return spec.getSpec().getIOConfig(); + return spec.getIoConfig(); } @VisibleForTesting 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 0f54232221bd..ef627a110c58 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 @@ -130,17 +130,17 @@ public KafkaSupervisor( protected RecordSupplier setupRecordSupplier() { return new KafkaRecordSupplier( - spec.getSpec().getIOConfig().getConsumerProperties(), + spec.getIoConfig().getConsumerProperties(), sortingMapper, - spec.getSpec().getIOConfig().getConfigOverrides(), - spec.getSpec().getIOConfig().isMultiTopic() + spec.getIoConfig().getConfigOverrides(), + spec.getIoConfig().isMultiTopic() ); } @Override protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) { - Integer taskCount = spec.getSpec().getIOConfig().getTaskCount(); + Integer taskCount = spec.getIoConfig().getTaskCount(); if (partitionId.isMultiTopicPartition()) { return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; } else { @@ -171,7 +171,7 @@ protected SeekableStreamSupervisorReportPayload creat boolean includeOffsets ) { - KafkaSupervisorIOConfig ioConfig = spec.getSpec().getIOConfig(); + KafkaSupervisorIOConfig ioConfig = spec.getIoConfig(); Map partitionLag = getRecordLagPerPartitionInLatestSequences(getHighestCurrentOffsets()); return new KafkaSupervisorReportPayload( spec.getId(), @@ -248,7 +248,7 @@ protected SeekableStreamIndexTaskIOConfig createUpdat SeekableStreamStartSequenceNumbers startSequenceNumbers = new SeekableStreamStartSequenceNumbers<>( - spec.getSpec().getIOConfig().getStream(), + spec.getIoConfig().getStream(), startingSequences, exclusiveStartSequenceNumberPartitions ); @@ -261,7 +261,7 @@ protected SeekableStreamIndexTaskIOConfig createUpdat SeekableStreamEndSequenceNumbers endSequenceNumbers = new SeekableStreamEndSequenceNumbers<>( - spec.getSpec().getIOConfig().getStream(), + spec.getIoConfig().getStream(), endingSequences ); @@ -278,15 +278,15 @@ protected SeekableStreamIndexTaskIOConfig createUpdat null, startSequenceNumbers, endSequenceNumbers, - spec.getSpec().getIOConfig().getConsumerProperties(), - spec.getSpec().getIOConfig().getPollTimeout(), + spec.getIoConfig().getConsumerProperties(), + spec.getIoConfig().getPollTimeout(), true, existingTaskGroup.getMinimumMessageTime(), existingTaskGroup.getMaximumMessageTime(), - spec.getSpec().getIOConfig().getInputFormat(), - spec.getSpec().getIOConfig().getConfigOverrides(), - spec.getSpec().getIOConfig().isMultiTopic(), - spec.getSpec().getIOConfig().getTaskDuration().getStandardMinutes() + spec.getIoConfig().getInputFormat(), + spec.getIoConfig().getConfigOverrides(), + spec.getIoConfig().isMultiTopic(), + spec.getIoConfig().getTaskDuration().getStandardMinutes() ); } @@ -617,7 +617,7 @@ protected String baseTaskName() @VisibleForTesting public KafkaSupervisorIOConfig getIoConfig() { - return spec.getSpec().getIOConfig(); + return spec.getIoConfig(); } @VisibleForTesting diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java index 513446b1c27b..c2aa56583092 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java @@ -30,7 +30,7 @@ public class KafkaSupervisorSpecBuilder { private String id; - private Boolean usePerpetuallyRunningTasks; + private Boolean usePersistentTasks; private final DataSchema.Builder dataSchema = new DataSchema.Builder(); private final KafkaIOConfigBuilder ioConfig = new KafkaIOConfigBuilder(); private final KafkaTuningConfigBuilder tuningConfig = new KafkaTuningConfigBuilder(); @@ -59,9 +59,9 @@ public KafkaSupervisorSpecBuilder withId(String id) return this; } - public KafkaSupervisorSpecBuilder withUsePerpetuallyRunningTasks(Boolean usePerpetuallyRunningTasks) + public KafkaSupervisorSpecBuilder withUsePersistentTasks(Boolean usePersistentTasks) { - this.usePerpetuallyRunningTasks = usePerpetuallyRunningTasks; + this.usePersistentTasks = usePersistentTasks; return this; } @@ -96,7 +96,7 @@ private KafkaSupervisorSpec build() ioConfig.build(), null, false, - usePerpetuallyRunningTasks, + usePersistentTasks, // Jackson injected params, not needed while posting a supervisor to the Overlord null, null, null, null, null, null, null, null, null ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 016e8c71060c..0a6d230e48c4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -188,7 +188,6 @@ public enum Status // - In possiblyPause(), when [shouldResume] is signalled, if [pauseRequested] has become false the pause loop ends, // [status] is changed to STARTING and [shouldResume] is signalled. private final Lock pauseLock = new ReentrantLock(); - private final Lock updateConfigLock = new ReentrantLock(); private final Condition hasPaused = pauseLock.newCondition(); private final Condition shouldResume = pauseLock.newCondition(); @@ -1773,7 +1772,7 @@ private Response updateTaskRunnerConfig(TaskConfigUpdateRequest newIoConfig = request.getIoConfig(); setIOConfig(newIoConfig); - createNewSequenceFromIoConfig(newIoConfig); + createNewSequenceFromIoConfig(newIoConfig, request.getLastOffsets()); supervisorSpecVersion = request.getSupervisorSpecVersion(); assignment = assignPartitions(recordSupplier); @@ -1808,22 +1807,37 @@ private void setIOConfig( * Creates new sequences for the ingestion process. It currently accepts the ioConfig given by the request as the correct offsets * and ignores the offsets it may have stored in currOffsets and endOffsets. */ - private void createNewSequenceFromIoConfig(SeekableStreamIndexTaskIOConfig ioConfig) + private void createNewSequenceFromIoConfig(SeekableStreamIndexTaskIOConfig ioConfig, + Map latestCommittedOffsets + ) throws IOException { Map partitionStartOffsets = ioConfig.getStartSequenceNumbers() .getPartitionSequenceNumberMap(); Map partitionEndSequences = ioConfig.getEndSequenceNumbers() .getPartitionSequenceNumberMap(); + SequenceMetadata lastSequenceMetadata = getLastSequenceMetadata(); + Map offsetsForLastPartitionAssignment = latestCommittedOffsets.entrySet() + .stream() + .filter(e -> lastSequenceMetadata.startOffsets.containsKey(e.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + if (lastSequenceMetadata.startOffsets.equals(offsetsForLastPartitionAssignment)) { + // This is the case where no data has been ingested since the last successfull checkpoint for these partitions. + // In this case, we'll mark the end offsets for this sequence same as start offsets and create a new sequence. + log.info("No new data ingested across any sibling tasks for this partition sequence since last checkpoint."); + lastSequenceMetadata.setEndOffsets(lastSequenceMetadata.startOffsets); + } final Set exclusiveStartPartitions = computeExclusiveStartPartitionsForSequence( partitionStartOffsets); + final SequenceMetadata newSequence = new SequenceMetadata<>( - sequences.isEmpty() ? 0 : getLastSequenceMetadata().getSequenceId() + 1, + sequences.isEmpty() ? 0 : lastSequenceMetadata.getSequenceId() + 1, StringUtils.format( "%s_%d", ioConfig.getBaseSequenceName(), - sequences.isEmpty() ? 0 : getLastSequenceMetadata().getSequenceId() + 1 + sequences.isEmpty() ? 0 : lastSequenceMetadata.getSequenceId() + 1 ), partitionStartOffsets, partitionEndSequences, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java index faa2167993a3..9528135e71a7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; +import java.util.Map; import java.util.Objects; /** @@ -31,17 +32,19 @@ public class TaskConfigUpdateRequest { private final SeekableStreamIndexTaskIOConfig ioConfig; - private final String supervisorSpecVersion; + private final Map lastOffsets; @JsonCreator public TaskConfigUpdateRequest( @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig, - @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion + @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion, + @JsonProperty("lastOffsets") Map lastOffsets ) { this.ioConfig = ioConfig; this.supervisorSpecVersion = supervisorSpecVersion; + this.lastOffsets = lastOffsets; } @JsonProperty @@ -56,6 +59,12 @@ public SeekableStreamIndexTaskIOConfig getI return ioConfig; } + @JsonProperty + public Map getLastOffsets() + { + return lastOffsets; + } + @Override public boolean equals(Object object) { 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 32aa4eb3c252..bcdc3c3ae662 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 @@ -793,7 +793,7 @@ private boolean sendConfigUpdatesToTasks( TaskGroup newTaskGroup = existingTaskGroup.withStartingSequences(newStartingSequences); for (String taskId : existingTaskGroup.taskIds()) { log.info("Updating config for task [%s] with partitions [%s]", taskId, partitionsForThisTask); - TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest<>(newIoConfig, spec.getVersion().get()); + TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest<>(newIoConfig, spec.getVersion().get(), latestTaskOffsetsOnPause); updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); } activelyReadingTaskGroups.put(taskGroupId, newTaskGroup); @@ -1975,7 +1975,7 @@ private List getCurrentParseErrors() // store a limited number of parse exceptions, keeping the most recent ones int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePersistentTasks()).getMaxSavedParseExceptions() * - spec.getSpec().getIOConfig().getTaskCount(); + spec.getIoConfig().getTaskCount(); parseErrorLimit = Math.min(parseErrorLimit, parseErrorsTreeSet.size()); final List limitedParseErrors = new ArrayList<>(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java index 48c53921c733..1a1c2c7a5f19 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java @@ -643,7 +643,7 @@ public void test_pauseAndCheckpointAsync_notAvailable() throws Exception @Test public void test_updateConfigAsync_success() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/config") @@ -660,7 +660,7 @@ public void test_updateConfigAsync_success() throws Exception @Test public void test_updateConfigAsync_httpError() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/config") @@ -680,7 +680,7 @@ public void test_updateConfigAsync_httpError() throws Exception @Test public void test_updateConfigAsync_notAvailable() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/config") @@ -695,7 +695,7 @@ public void test_updateConfigAsync_notAvailable() throws Exception @Test public void test_updateConfigAsync_closed() throws Exception { - final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null); + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); serviceClient.expectAndThrow( new RequestBuilder(HttpMethod.POST, "/config") diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java index bad2e95747db..7ff448833614 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java @@ -31,13 +31,13 @@ public void testTaskConfigUpdateRequest() SeekableStreamIndexTaskIOConfig mockIoConfig = EasyMock.createMock(SeekableStreamIndexTaskIOConfig.class); EasyMock.replay(mockIoConfig); - TaskConfigUpdateRequest request = new TaskConfigUpdateRequest(mockIoConfig, null); + TaskConfigUpdateRequest request = new TaskConfigUpdateRequest(mockIoConfig, null, null); Assert.assertEquals(mockIoConfig, request.getIoConfig()); - TaskConfigUpdateRequest nullRequest = new TaskConfigUpdateRequest(null, null); + TaskConfigUpdateRequest nullRequest = new TaskConfigUpdateRequest(null, null, null); Assert.assertNull(nullRequest.getIoConfig()); - TaskConfigUpdateRequest request2 = new TaskConfigUpdateRequest(mockIoConfig, null); + TaskConfigUpdateRequest request2 = new TaskConfigUpdateRequest(mockIoConfig, null, null); Assert.assertEquals(request, request2); Assert.assertEquals(request.hashCode(), request2.hashCode()); From 799c547a77baa79e540edd01c7e13de6186b1bcc Mon Sep 17 00:00:00 2001 From: Uddeshya Singh Date: Sun, 28 Sep 2025 16:30:44 +0530 Subject: [PATCH 46/46] Complete all embedded test additions --- .../indexing/KafkaTaskAutoScalingTest.java | 227 ++++++++++++++++++ .../indexing/KafkaTaskScalingTest.java | 22 +- .../kafka/supervisor/KafkaSupervisorTest.java | 12 +- .../supervisor/KinesisSupervisorTest.java | 10 +- .../cluster/overlord/FaultyLagAggregator.java | 65 ++++- .../supervisor/SeekableStreamSupervisor.java | 120 ++++----- 6 files changed, 375 insertions(+), 81 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.java new file mode 100644 index 000000000000..2c4ae7d7007f --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.java @@ -0,0 +1,227 @@ +/* + * 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.data.input.impl.TimestampSpec; +import org.apache.druid.guice.ClusterTestingModule; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.LagBasedAutoScalerConfigBuilder; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; +import org.apache.druid.testing.cluster.overlord.FaultyLagAggregator; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedClusterApis; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +import org.apache.druid.testing.embedded.EmbeddedRouter; +import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +/** + * Embedded test to verify task scaling behaviour of {@code KafkaSupervisor} ingesting from a custom kafka topic. + */ +@SuppressWarnings("resource") +public class KafkaTaskAutoScalingTest extends EmbeddedClusterTestBase +{ + private static final String TOPIC = EmbeddedClusterApis.createTestDatasourceName(); + + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + + private KafkaResource kafkaServer; + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + createTopicWithPartitions(TOPIC, 10); + } + + @Override + public void stop() + { + deleteTopic(TOPIC); + super.stop(); + } + }; + + indexer.addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.server.http.numThreads", "50") + .addProperty("druid.unsafe.cluster.testing", "true"); + + cluster.addExtension(KafkaIndexTaskModule.class) + .addExtension(ClusterTestingModule.class) + .addResource(kafkaServer) + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(historical) + .addServer(new EmbeddedRouter()) + .useLatchableEmitter(); + + return cluster; + } + + @Test + @Timeout(50) + public void test_supervisorTasksScalesOutAndScalesIn_withPersistentTasksAndAutoScaler() throws Exception + { + final int initialTaskCount = 1; + final int taskCountMax = 3; + final int taskCountMin = 1; + + AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() + .withLagCollectionIntervalMillis(500) + .withLagCollectionRangeMillis(1000) + .withEnableTaskAutoScaler(true) + .withScaleActionPeriodMillis(2000) + .withScaleActionStartDelayMillis(1000) + .withScaleOutThreshold(100) + .withScaleInThreshold(1) + .withTaskCountMin(taskCountMin) + .withTaskCountMax(taskCountMax) + .withTaskCountStart(initialTaskCount) + .withTriggerScaleOutFractionThreshold(0.001) + .withTriggerScaleInFractionThreshold(0.01) + .withScaleOutStep(1) + .withScaleInStep(1) + .withMinTriggerScaleActionFrequencyMillis(3000) + .withStopTaskCountRatio(1.0) + .build(); + + String supervisorId = dataSource + "_scale_out_and_in_to_zero"; + String controllerId = "artificial-lag-controller"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpecWithControlledLag( + supervisorId, + controllerId, + initialTaskCount, + autoScalerConfig, + true + ); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + FaultyLagAggregator.injectLag(controllerId, 100000L); + + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("task/autoScaler/scaleActionTime"), + agg -> agg.hasCountAtLeast(2) + ); + + Thread.sleep(5000); + + String runningTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", + dataSource + ); + int runningTasks = Integer.parseInt(runningTaskCountResult); + + Assertions.assertEquals( + taskCountMax, + runningTasks, + String.format("Expected %d tasks to be running after scale up, but found %d", taskCountMax, runningTasks) + ); + + FaultyLagAggregator.injectLag(controllerId, 0L); + + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("task/autoScaler/scaleActionTime"), + agg -> agg.hasCountAtLeast(2) + ); + Thread.sleep(10000); + + // ensure everything has shutdown. + runningTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", + dataSource + ); + String successTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'SUCCESS'", + dataSource + ); + runningTasks = Integer.parseInt(runningTaskCountResult); + final int successTasks = Integer.parseInt(successTaskCountResult); + + Assertions.assertEquals( + taskCountMin, + runningTasks, + String.format("Expected %d task to be running after scale down, but found %d", runningTasks, taskCountMin) + ); + + int shutDownTasksExpected = taskCountMax - taskCountMin; + + Assertions.assertTrue( + successTasks >= shutDownTasksExpected, + String.format("Expected at least %d task to be successfully completed, but found %d", shutDownTasksExpected, successTasks) + ); + + // Cleanup + FaultyLagAggregator.clearAllInjectedLag(); + cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); + } + + private KafkaSupervisorSpec createSupervisorSpecWithControlledLag( + String supervisorId, + String controllerId, + int taskCount, + AutoScalerConfig autoScalerConfig, + boolean usePersistentTasks + ) + { + return MoreResources.Supervisor.KAFKA_JSON + .get() + .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) + .withTuningConfig(tuningConfig -> tuningConfig + .withMaxRowsPerSegment(1000) + .withWorkerThreads(10)) + .withIoConfig( + ioConfig -> ioConfig + .withConsumerProperties(kafkaServer.consumerProperties()) + .withTaskCount(taskCount) + .withTaskDuration(Period.millis(30000)) + .withAutoScalerConfig(autoScalerConfig) + .withLagAggregator(new FaultyLagAggregator(1, controllerId)) + ) + .withId(supervisorId) + .withUsePersistentTasks(usePersistentTasks) + .build(dataSource, TOPIC); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java index 0e778a631cc2..290c20d18fea 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java @@ -77,10 +77,8 @@ public void stop() } }; - indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") - .addProperty("druid.worker.capacity", "10"); - overlord.addProperty("druid.server.http.numThreads", "50") - .addProperty("druid.manager.segments.pollDuration", "PT0.1s"); + indexer.addProperty("druid.worker.capacity", "10"); + overlord.addProperty("druid.server.http.numThreads", "50"); cluster.addExtension(KafkaIndexTaskModule.class) .addResource(kafkaServer) .addServer(coordinator) @@ -110,7 +108,8 @@ public void test_supervisorTasksFinish_withNoDataAndShortTaskDuration() try { Thread.sleep(2000); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); Assertions.fail("Test interrupted"); } @@ -146,7 +145,8 @@ public void test_supervisorTasksDontFinish_withPersistentTasks() try { Thread.sleep(10000); - } catch (InterruptedException e) { + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); Assertions.fail("Test interrupted"); } @@ -212,7 +212,8 @@ public void test_supervisorTasksScalesIn_withPersistentTasksAndAutoScaler() try { Thread.sleep(2000); // Wait for a few seconds for the tasks to scale in - } catch (InterruptedException e) { + } + catch (InterruptedException e) { Thread.currentThread().interrupt(); Assertions.fail("Test interrupted"); } @@ -231,13 +232,6 @@ public void test_supervisorTasksScalesIn_withPersistentTasksAndAutoScaler() ); } - @Test - @Timeout(20) - public void tesT_supervisorTasksScalesOutAndScalesIn_withPersistentTasksAndAutoScaler() - { - - } - private KafkaSupervisorSpec createSupervisorSpec( String supervisorId, int taskCount, 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 e3816763bb46..bab2f66b3156 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 @@ -4973,11 +4973,11 @@ public void testIsTaskCurrent() replayAll(); - Assert.assertTrue(supervisor.isTaskCurrent(42, "id0", taskMap)); - Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap)); + Assert.assertTrue(supervisor.isTaskCurrent(42, "id0", taskMap, "")); + Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap, "")); verifyAll(); } @@ -6093,7 +6093,7 @@ public TestableKafkaSupervisorWithCustomIsTaskCurrent( } @Override - public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap) + public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap, String version) { return isTaskCurrentReturn; } 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 4a83cf81e393..9347dc6d7694 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 @@ -4094,10 +4094,10 @@ public void testIsTaskCurrent() replayAll(); - Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap)); + Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap, "")); verifyAll(); } @@ -5772,7 +5772,7 @@ private class TestableKinesisSupervisorWithCustomIsTaskCurrent extends TestableK } @Override - public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap) + public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap, String version) { return isTaskCurrentReturn; } diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java index 90d9f3ac420e..c87d436114a9 100644 --- a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java @@ -25,29 +25,50 @@ import org.apache.druid.indexing.seekablestream.supervisor.LagAggregator; import org.apache.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** * Implementation of {@link LagAggregator} that supports the following: *
      *
    • Specify a {@code multiplier} to amplify the lag observed by the Overlord * for a given supervisor.
    • + *
    • Support externally controllable lag injection for testing auto-scaling scenarios.
    • *
    */ public class FaultyLagAggregator implements LagAggregator { private static final Logger log = new Logger(FaultyLagAggregator.class); + // Global registry for external control from tests + private static final ConcurrentHashMap INJECTED_LAG_REGISTRY = new ConcurrentHashMap<>(); + private final int lagMultiplier; + private final String clientId; private final LagAggregator delegate = LagAggregator.DEFAULT; + public FaultyLagAggregator(int lagMultiplier) + { + this.lagMultiplier = lagMultiplier; + this.clientId = null; + log.info("Multiplying lags by factor[%d].", lagMultiplier); + } + @JsonCreator public FaultyLagAggregator( - @JsonProperty("lagMultiplier") int lagMultiplier + @JsonProperty("lagMultiplier") int lagMultiplier, + @JsonProperty("clientId") @Nullable String clientId ) { this.lagMultiplier = lagMultiplier; - log.info("Multiplying lags by factor[%d].", lagMultiplier); + this.clientId = clientId; + + if (clientId != null) { + log.info("Controllable faulty lag aggregator with extenral ID[%s]", clientId); + } else { + log.info("Multiplying lags by factor[%d].", lagMultiplier); + } } @JsonProperty @@ -56,8 +77,36 @@ public int getLagMultiplier() return lagMultiplier; } + @JsonProperty + @Nullable + public String getClientId() + { + return clientId; + } + @Override public LagStats aggregate(Map partitionLags) + { + if (clientId != null && INJECTED_LAG_REGISTRY.containsKey(clientId)) { + return aggregateWithInjectedLag(partitionLags); + } else { + return aggregateWithMultiplier(partitionLags); + } + } + + private LagStats aggregateWithInjectedLag(Map partitionLags) + { + long injectedLag = INJECTED_LAG_REGISTRY.get(clientId); + + log.debug("Using injected lag[%d] for controller[%s]", injectedLag, clientId); + + // Return fixed lag values regardless of actual partition lags + long totalLag = partitionLags.isEmpty() ? injectedLag : injectedLag * partitionLags.size(); + + return new LagStats(injectedLag, totalLag, injectedLag); + } + + private LagStats aggregateWithMultiplier(Map partitionLags) { LagStats originalAggregate = delegate.aggregate(partitionLags); return new LagStats( @@ -66,4 +115,16 @@ public LagStats aggregate(Map partition originalAggregate.getAvgLag() * getLagMultiplier() ); } + + public static void injectLag(String controllerId, long lag) + { + log.info("Injecting lag[%d] for controller[%s]", lag, controllerId); + INJECTED_LAG_REGISTRY.put(controllerId, lag); + } + + public static void clearAllInjectedLag() + { + log.info("Clearing all injected lag"); + INJECTED_LAG_REGISTRY.clear(); + } } 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 bcdc3c3ae662..72b390cef288 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 @@ -1046,6 +1046,7 @@ public String getType() protected class CheckpointNotice implements Notice { + private static final int DEFAULT_REENTRY_DELAY = 1500; private final int taskGroupId; private final SeekableStreamDataSourceMetadata checkpointMetadata; private static final String TYPE = "checkpoint_notice"; @@ -1097,7 +1098,7 @@ public void handle() throws ExecutionException, InterruptedException // It might be possible that the task has not been discovered to the taskgroup yet and have received a checkpoint before hand, // For now, I will attempt to repush the checkpoint request in the handler. log.warn("New checkpoint is null for taskGroup [%s]", taskGroupId); - addNotice(this); + scheduledExec.schedule(() -> addNotice(this), DEFAULT_REENTRY_DELAY, TimeUnit.MILLISECONDS); } } } @@ -2419,6 +2420,19 @@ private void discoverTasks() throws ExecutionException, InterruptedException taskCount++; @SuppressWarnings("unchecked") final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; + SeekableStreamIndexTaskIOConfig taskIoConfig; + final String taskSpecVersion; + if (spec.usePersistentTasks()) { + // Fetch the current ioConfig being run on the task + final TaskConfigResponse runningConfig = FutureUtils.get( + taskClient.getTaskConfigAsync(task.getId()), true); + taskIoConfig = runningConfig.getIoConfig(); + taskSpecVersion = runningConfig.getSupervisorSpecVersion(); + } else { + taskSpecVersion = ""; + taskIoConfig = seekableStreamIndexTask.getIOConfig(); + } + final String taskId = task.getId(); // Check if the task has any inactive partitions. If so, terminate the task. Even if some of the @@ -2426,10 +2440,10 @@ private void discoverTasks() throws ExecutionException, InterruptedException // to more rapidly ensure that all active partitions are evenly distributed and being read, and to avoid // having to map expired partitions which are no longer tracked in partitionIds to a task group. if (supportsPartitionExpiration()) { - Set taskPartitions = seekableStreamIndexTask.getIOConfig() - .getStartSequenceNumbers() - .getPartitionSequenceNumberMap() - .keySet(); + Set taskPartitions = taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .keySet(); Set inactivePartitionsInTask = Sets.difference( taskPartitions, new HashSet<>(partitionIds) @@ -2452,11 +2466,11 @@ private void discoverTasks() throws ExecutionException, InterruptedException // state, we will permit it to complete even if it doesn't match our current partition allocation to support // seamless schema migration. - Iterator it = seekableStreamIndexTask.getIOConfig() - .getStartSequenceNumbers() - .getPartitionSequenceNumberMap() - .keySet() - .iterator(); + Iterator it = taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .keySet() + .iterator(); final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); if (taskGroupId != null) { @@ -2480,19 +2494,19 @@ public Boolean apply(Pair addDiscoveredTaskToPendingCompletionTaskGroups( - getTaskGroupIdForPartition( - partition), - taskId, - seekableStreamIndexTask.getIOConfig() - .getStartSequenceNumbers() - .getPartitionSequenceNumberMap() - )); + taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .keySet() + .forEach( + partition -> addDiscoveredTaskToPendingCompletionTaskGroups( + getTaskGroupIdForPartition( + partition), + taskId, + taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + )); // update partitionGroups with the publishing task's sequences (if they are greater than what is // existing) so that the next tasks will start reading from where this task left off. @@ -2532,10 +2546,10 @@ public Boolean apply(Pair activeTaskMap) + public boolean isTaskCurrent(int taskGroupId, String taskId, Map activeTaskMap, String version) { final Task genericTask; @@ -3060,7 +3074,7 @@ public boolean isTaskCurrent(int taskGroupId, String taskId, Map a // If the persistent tasks are present, we should use the version. if (spec.usePersistentTasks()) { - return isPersistentTaskCurrent(task); + return version.equals(spec.getVersion().get()); } // We recompute the sequence name hash for the supervisor's own configuration and compare this to the hash created @@ -3097,22 +3111,6 @@ public boolean isTaskCurrent(int taskGroupId, String taskId, Map a } } - /** - * Verifies whether the running config version of the persistent task matches one in current supervisor spec. - */ - private boolean isPersistentTaskCurrent(SeekableStreamIndexTask task) - { - try { - final String currentVersion = spec.getVersion().get(); - final TaskConfigResponse runningConfig = FutureUtils.get( - taskClient.getTaskConfigAsync(task.getId()), true); - return currentVersion.equals(runningConfig.getSupervisorSpecVersion()); - } - catch (Exception e) { - log.error(e, "Could not fetch running config for task[%s]", task.getId()); - return false; - } - } @VisibleForTesting public String generateSequenceName( @@ -4083,9 +4081,10 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep Entry task = iTasks.next(); String taskId = task.getKey(); TaskData taskData = task.getValue(); + final String version = spec.usePersistentTasks() ? getRunningSpecVersionOnTask(taskId) : spec.getVersion().get(); // stop and remove bad tasks from the task group - if (!isTaskCurrent(groupId, taskId, activeTaskMap)) { + if (!isTaskCurrent(groupId, taskId, activeTaskMap, version)) { log.info("Stopping task[%s] as it does not match the expected sequence range and ingestion spec.", taskId); futures.add(stopTask(taskId, false)); iTasks.remove(); @@ -4117,6 +4116,19 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep coalesceAndAwait(futures); } + private String getRunningSpecVersionOnTask(String taskId) + { + try { + final TaskConfigResponse runningConfig = FutureUtils.get( + taskClient.getTaskConfigAsync(taskId), true); + return runningConfig.getSupervisorSpecVersion(); + } + catch (InterruptedException | ExecutionException e) { + log.warn("Interrupted while fetching running spec version for task[%s]", taskId); + return ""; + } + } + private void checkIfStreamInactiveAndTurnSupervisorIdle() { if (!idleConfig.isEnabled() || spec.isSuspended()) {