From e5a05120f77480f10bbff4becbc093fdfc58bf29 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Wed, 17 Dec 2025 18:13:37 +0200 Subject: [PATCH 01/20] Inject new task count calculation during the rollover --- .../indexing/TaskScaleDownRolloverTest.java | 232 ++++++++++++++++++ .../supervisor/SupervisorManager.java | 5 + .../supervisor/SeekableStreamSupervisor.java | 168 ++++++++++--- .../autoscaler/CostBasedAutoScaler.java | 29 ++- .../autoscaler/CostBasedAutoScalerTest.java | 18 +- .../autoscaler/SupervisorTaskAutoScaler.java | 11 + 6 files changed, 425 insertions(+), 38 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java new file mode 100644 index 000000000000..b29a5c99f2bd --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java @@ -0,0 +1,232 @@ +/* + * 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.seekablestream.supervisor.autoscaler.CostBasedAutoScalerConfig; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.DruidMetrics; +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.apache.kafka.clients.producer.ProducerRecord; +import org.hamcrest.Matchers; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Seconds; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler.OPTIMAL_TASK_COUNT_METRIC; + +@SuppressWarnings("SameParameterValue") +public class TaskScaleDownRolloverTest extends EmbeddedClusterTestBase +{ + private static final String TOPIC = EmbeddedClusterApis.createTestDatasourceName(); + private static final String EVENT_TEMPLATE = "{\"timestamp\":\"%s\",\"dimension\":\"value%d\",\"metric\":%d}"; + private static final int PARTITION_COUNT = 50; + + private final EmbeddedIndexer indexer = new EmbeddedIndexer(); + private final EmbeddedOverlord overlord = new EmbeddedOverlord(); + private KafkaResource kafkaServer; + + @Override + public EmbeddedDruidCluster createCluster() + { + final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); + + kafkaServer = new KafkaResource() + { + @Override + public void start() + { + super.start(); + createTopicWithPartitions(TOPIC, PARTITION_COUNT); + produceRecordsToKafka(500, 1); + } + + @Override + public void stop() + { + deleteTopic(TOPIC); + super.stop(); + } + }; + + // Increase worker capacity to handle more tasks + indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.worker.capacity", "60"); + + overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.manager.segments.pollDuration", "PT0.1s"); + + cluster.useLatchableEmitter() + .useDefaultTimeoutForLatchableEmitter(20) + .addServer(new EmbeddedCoordinator()) + .addServer(overlord) + .addServer(indexer) + .addServer(new EmbeddedRouter()) + .addServer(new EmbeddedBroker()) + .addServer(new EmbeddedHistorical()) + .addExtension(KafkaIndexTaskModule.class) + .addCommonProperty("druid.monitoring.emissionPeriod", "PT1s") + .addResource(kafkaServer); + + return cluster; + } + + /** + * Tests that scale down happen during task rollover via checkTaskDuration(). + * + *

Test flow:

+ *
    + *
  1. Start supervisor with 20 tasks and 50 partitions, minimal data (500 records)
  2. + *
  3. Wait for initial tasks to start running
  4. + *
  5. Wait for the first task rollover to complete (task duration is 10 seconds)
  6. + *
  7. Verify that after rollover, fewer tasks are running due to cost-based autoscaler (no ingestion at all)
  8. + *
+ * + *

Scale down during rollover is triggered in {@code SeekableStreamSupervisor.checkTaskDuration()} + * when all task groups have rolled over and the autoscaler recommends a lower task count.

+ */ + @SuppressWarnings("resource") + @Test + @Timeout(300) + void test_scaleDownDuringTaskRollover() + { + final String superId = dataSource + "_super"; + final int initialTaskCount = 10; + + final CostBasedAutoScalerConfig autoScalerConfig = CostBasedAutoScalerConfig + .builder() + .enableTaskAutoScaler(true) + .taskCountMin(1) + .taskCountMax(10) + .taskCountStart(initialTaskCount) + .scaleActionPeriodMillis(2000) + .minTriggerScaleActionFrequencyMillis(2000) + // High idle weight ensures scale-down when tasks are mostly idle (little data to process) + .lagWeight(0.1) + .idleWeight(0.9) + .build(); + + final KafkaSupervisorSpec spec = createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig, initialTaskCount); + + // Submit the supervisor + Assertions.assertEquals(superId, cluster.callApi().postSupervisor(spec)); + + // Wait for at least one task running for the datasource managed by the supervisor. + overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/run/time") + .hasDimension(DruidMetrics.DATASOURCE, dataSource)); + + // Wait for autoscaler to emit metric indicating scale-down, it should be just less than the current task count. + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC) + .hasValueMatching(Matchers.lessThan((long) initialTaskCount))); + + // Wait for tasks to complete (first rollover) + overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/action/success/count")); + + // Wait for the task running for the datasource managed by a supervisor. + overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/run/time") + .hasDimension(DruidMetrics.DATASOURCE, dataSource)); + + // After rollover, verify that the running task count has decreased + // The autoscaler should have recommended fewer tasks due to high idle time + final int postRolloverRunningTasks = cluster.callApi().getTaskCount("running", dataSource); + + Assertions.assertTrue( + postRolloverRunningTasks < initialTaskCount, + StringUtils.format( + "Expected running task count to decrease after rollover. Initial: %d, After rollover: %d", + initialTaskCount, + postRolloverRunningTasks + ) + ); + + // Suspend the supervisor to clean up + cluster.callApi().postSupervisor(spec.createSuspendedSpec()); + } + + private KafkaSupervisorSpec createKafkaSupervisorWithAutoScaler( + String supervisorId, + CostBasedAutoScalerConfig autoScalerConfig, + int taskCount + ) + { + return MoreResources.Supervisor.KAFKA_JSON + .get() + .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) + .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(100)) + .withIoConfig( + ioConfig -> ioConfig + .withConsumerProperties(kafkaServer.consumerProperties()) + .withTaskCount(taskCount) + .withTaskDuration(Seconds.parseSeconds("PT10S").toPeriod()) + .withAutoScalerConfig(autoScalerConfig) + ) + .withId(supervisorId) + .build(dataSource, TOPIC); + } + + private void produceRecordsToKafka(int recordCount, int iterations) + { + int recordCountPerSlice = recordCount / iterations; + int counter = 0; + for (int i = 0; i < iterations; i++) { + DateTime timestamp = DateTime.now(DateTimeZone.UTC); + List> records = IntStream + .range(counter, counter + recordCountPerSlice) + .mapToObj(k -> new ProducerRecord( + TOPIC, + k % PARTITION_COUNT, + null, + StringUtils.format(EVENT_TEMPLATE, timestamp, k, k) + .getBytes(StandardCharsets.UTF_8) + ) + ) + .collect(Collectors.toList()); + + kafkaServer.produceRecordsToTopic(records); + try { + Thread.sleep(100L); + counter += recordCountPerSlice; + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } +} 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 21d2a6265011..ccc31ef639e4 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 @@ -504,6 +504,11 @@ private boolean createAndStartSupervisorInternal(SupervisorSpec spec, boolean pe supervisor = spec.createSupervisor(); autoscaler = spec.createAutoscaler(supervisor); + // Wire autoscaler back to supervisor for rollover-based scale-down + if (supervisor instanceof SeekableStreamSupervisor && autoscaler != null) { + ((SeekableStreamSupervisor) supervisor).setTaskAutoScaler(autoscaler); + } + supervisor.start(); if (autoscaler != null) { autoscaler.start(); 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 90adc68c799a..2259ff1aa380 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 @@ -63,6 +63,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; +import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler; import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; @@ -468,15 +469,19 @@ public void handle() ); return; } - log.debug("PendingCompletionTaskGroups is [%s] for supervisor[%s] for dataSource[%s]", pendingCompletionTaskGroups, - supervisorId, - dataSource + log.debug( + "PendingCompletionTaskGroups is [%s] for supervisor[%s] for dataSource[%s]", pendingCompletionTaskGroups, + supervisorId, + dataSource ); final Integer desiredTaskCount = computeDesiredTaskCount.call(); ServiceMetricEvent.Builder event = ServiceMetricEvent.builder() .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) .setDimension(DruidMetrics.DATASOURCE, dataSource) - .setDimension(DruidMetrics.STREAM, getIoConfig().getStream()); + .setDimension( + DruidMetrics.STREAM, + getIoConfig().getStream() + ); for (CopyOnWriteArrayList list : pendingCompletionTaskGroups.values()) { if (!list.isEmpty()) { log.info( @@ -596,7 +601,12 @@ private boolean changeTaskCount(int desiredActiveTaskCount) AUTOSCALER_SCALING_TIME_METRIC, scaleActionStopwatch.millisElapsed() )); - log.info("Changed taskCount to [%s] for supervisor[%s] for dataSource[%s].", desiredActiveTaskCount, supervisorId, dataSource); + log.info( + "Changed taskCount to [%s] for supervisor[%s] for dataSource[%s].", + desiredActiveTaskCount, + supervisorId, + dataSource + ); return true; } } @@ -610,11 +620,20 @@ private void changeTaskCountInIOConfig(int desiredActiveTaskCount) MetadataSupervisorManager metadataSupervisorManager = supervisorManager.get().getMetadataSupervisorManager(); metadataSupervisorManager.insert(supervisorId, spec); } else { - log.error("supervisorManager is null in taskMaster, skipping scale action for supervisor[%s] for dataSource[%s].", supervisorId, dataSource); + log.error( + "supervisorManager is null in taskMaster, skipping scale action for supervisor[%s] for dataSource[%s].", + supervisorId, + dataSource + ); } } catch (Exception e) { - log.error(e, "Failed to sync taskCount to MetaStorage for supervisor[%s] for dataSource[%s].", supervisorId, dataSource); + log.error( + e, + "Failed to sync taskCount to MetaStorage for supervisor[%s] for dataSource[%s].", + supervisorId, + dataSource + ); } } @@ -740,7 +759,11 @@ public void handle() for (Integer taskGroupId : taskGroupIds) { TaskGroup taskGroup = activelyReadingTaskGroups.getOrDefault(taskGroupId, null); if (taskGroup == null) { - log.info("Tried to stop task group [%d] for supervisor [%s] that wasn't actively reading.", taskGroupId, supervisorId); + log.info( + "Tried to stop task group [%d] for supervisor [%s] that wasn't actively reading.", + taskGroupId, + supervisorId + ); continue; } log.info("Task group [%d] for supervisor [%s] will handoff early.", taskGroupId, supervisorId); @@ -826,8 +849,9 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) log.warn("Ignoring checkpoint request because taskGroup[%d] is inactive", taskGroupId); return false; } else { - throw new ISE("Cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId, - activelyReadingTaskGroups + throw new ISE( + "Cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId, + activelyReadingTaskGroups ); } } @@ -885,7 +909,7 @@ public String getType() /** * Tag for identifying this supervisor in thread-names, listeners, etc. tag = (type + supervisorId). - */ + */ private final String supervisorTag; private final TaskInfoProvider taskInfoProvider; private final RowIngestionMetersFactory rowIngestionMetersFactory; @@ -926,6 +950,12 @@ public String getType() private volatile boolean lifecycleStarted = false; private final ServiceEmitter emitter; + /** + * Reference to the autoscaler, used for rollover-based scale-down decisions. + * Wired by {@link SupervisorManager} after supervisor creation. + */ + private volatile SupervisorTaskAutoScaler taskAutoScaler; + // snapshots latest sequences from the stream to be verified in the next run cycle of inactive stream check private final Map previousSequencesFromStream = new HashMap<>(); private long lastActiveTimeMillis; @@ -1200,6 +1230,7 @@ public void reset(@Nullable final DataSourceMetadata dataSourceMetadata) * Reset offsets with provided dataSource metadata. Validates {@code resetDataSourceMetadata}, * creates a {@code ResetOffsetsNotice} with the metadata and adds it to the notice queue. The resulting stored offsets * is a union of existing checkpointed offsets and provided offsets. + * * @param resetDataSourceMetadata required datasource metadata with offsets to reset. * @throws DruidException if any metadata attribute doesn't match the supervisor's. */ @@ -1306,14 +1337,19 @@ public void tryInit() if (log.isDebugEnabled()) { log.debug( "Handled notice[%s] from notices queue in [%d] ms, " - + "current notices queue size [%d] for supervisor[%s] for datasource[%s].", + + "current notices queue size [%d] for supervisor[%s] for datasource[%s].", noticeType, noticeHandleTime.millisElapsed(), getNoticesQueueSize(), supervisorId, dataSource ); } } catch (Throwable e) { stateManager.recordThrowableEvent(e); - log.makeAlert(e, "Supervisor[%s] for datasource[%s] failed to handle notice", supervisorId, dataSource) + log.makeAlert( + e, + "Supervisor[%s] for datasource[%s] failed to handle notice", + supervisorId, + dataSource + ) .addData("noticeClass", notice.getClass().getSimpleName()) .emit(); } @@ -1783,10 +1819,17 @@ public void runInternal() stateManager.recordThrowableEvent(e); if (e instanceof StreamException) { // When a StreamException is thrown, the error message is more useful than the stack trace in telling what's wrong. - log.makeAlert("Exception in supervisor run loop for supervisor[%s] for dataSource[%s]: [%s]", - supervisorId, dataSource, e.getMessage()).emit(); + log.makeAlert( + "Exception in supervisor run loop for supervisor[%s] for dataSource[%s]: [%s]", + supervisorId, dataSource, e.getMessage() + ).emit(); } else { - log.makeAlert(e, "Exception in supervisor run loop for supervisor[%s] for dataSource[%s]", supervisorId, dataSource).emit(); + log.makeAlert( + e, + "Exception in supervisor run loop for supervisor[%s] for dataSource[%s]", + supervisorId, + dataSource + ).emit(); } } finally { @@ -1858,7 +1901,12 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) if (dataSourceMetadata == null) { // Reset everything boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(supervisorId); - log.info("Reset supervisor[%s] for dataSource[%s] - dataSource metadata entry deleted? [%s]", supervisorId, dataSource, result); + log.info( + "Reset supervisor[%s] for dataSource[%s] - dataSource metadata entry deleted? [%s]", + supervisorId, + dataSource, + result + ); activelyReadingTaskGroups.values() .forEach(group -> killTasksInGroup( group, @@ -1874,7 +1922,12 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) dataSourceMetadata.getClass() ); } - log.info("Reset supervisor[%s] for dataSource[%s] with metadata[%s]", supervisorId, dataSource, dataSourceMetadata); + log.info( + "Reset supervisor[%s] for dataSource[%s] with metadata[%s]", + supervisorId, + dataSource, + dataSourceMetadata + ); // Reset only the partitions in dataSourceMetadata if it has not been reset yet @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = @@ -1931,7 +1984,10 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) } else { final DataSourceMetadata newMetadata = currentMetadata.minus(resetMetadata); try { - metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(supervisorId, newMetadata); + metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata( + supervisorId, + newMetadata + ); } catch (IOException e) { log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); @@ -1972,11 +2028,17 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) * existing checkpointed offsets and provided offsets; any checkpointed offsets not specified in the metadata will be * preserved as-is. If checkpoints don't exist, the provided reset datasource metdadata will be inserted into * the metadata storage. Once the offsets are reset, any active tasks serving the partition offsets will be restarted. + * * @param dataSourceMetadata Required reset data source metdata. Assumed that the metadata is validated. */ public void resetOffsetsInternal(@Nonnull final DataSourceMetadata dataSourceMetadata) { - log.info("Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s]", supervisorId, dataSource, dataSourceMetadata); + log.info( + "Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s]", + supervisorId, + dataSource, + dataSourceMetadata + ); @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = @@ -1985,7 +2047,12 @@ public void resetOffsetsInternal(@Nonnull final DataSourceMetadata dataSourceMet final boolean metadataUpdateSuccess; final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(supervisorId); if (metadata == null) { - log.info("Checkpointed metadata in null for supervisor[%s] for dataSource[%s] - inserting metadata[%s]", supervisorId, dataSource, resetMetadata); + log.info( + "Checkpointed metadata in null for supervisor[%s] for dataSource[%s] - inserting metadata[%s]", + supervisorId, + dataSource, + resetMetadata + ); metadataUpdateSuccess = indexerMetadataStorageCoordinator.insertDataSourceMetadata(supervisorId, resetMetadata); } else { if (!checkSourceMetadataMatch(metadata)) { @@ -1998,18 +2065,35 @@ public void resetOffsetsInternal(@Nonnull final DataSourceMetadata dataSourceMet final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; final DataSourceMetadata newMetadata = currentMetadata.plus(resetMetadata); - log.info("Current checkpointed metadata[%s], new metadata[%s] for supervisor[%s] for dataSource[%s]", currentMetadata, newMetadata, supervisorId, dataSource); + log.info( + "Current checkpointed metadata[%s], new metadata[%s] for supervisor[%s] for dataSource[%s]", + currentMetadata, + newMetadata, + supervisorId, + dataSource + ); try { metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(supervisorId, newMetadata); } catch (IOException e) { - log.error("Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s] failed [%s]", supervisorId, dataSource, newMetadata, e.getMessage()); + log.error( + "Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s] failed [%s]", + supervisorId, + dataSource, + newMetadata, + e.getMessage() + ); throw new RuntimeException(e); } } if (!metadataUpdateSuccess) { - throw new ISE("Unable to reset metadata[%s] for supervisor[%s] for dataSource[%s]", supervisorId, dataSource, dataSourceMetadata); + throw new ISE( + "Unable to reset metadata[%s] for supervisor[%s] for dataSource[%s]", + supervisorId, + dataSource, + dataSourceMetadata + ); } resetMetadata.getSeekableStreamSequenceNumbers() @@ -2319,7 +2403,12 @@ public Boolean apply(Pair 0 && rolloverTaskCount < ioConfig.getTaskCount()) { + log.info("Cost-based autoscaler recommends scaling down to [%d] tasks during rollover", rolloverTaskCount); + changeTaskCountInIOConfig(rolloverTaskCount); + // Here force reset the supervisor state to be re-calculated on the next iteration of runInternal() call. + // This seems the best way to inject task amount recalculation during the rollover. + partitionGroups.clear(); + partitionOffsets.clear(); + partitionIds.clear(); + } + } } private DateTime computeEarliestTaskStartTime(TaskGroup group) @@ -4330,6 +4432,15 @@ public SeekableStreamSupervisorIOConfig getIoConfig() return ioConfig; } + /** + * Sets the autoscaler reference for rollover-based scale-down decisions. + * Called by {@link SupervisorManager} after supervisor creation. + */ + public void setTaskAutoScaler(@Nullable SupervisorTaskAutoScaler taskAutoScaler) + { + this.taskAutoScaler = taskAutoScaler; + } + @Override public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) { @@ -4688,9 +4799,10 @@ protected void emitLag() // Try emitting lag even with stale metrics provided that none of the partitions has negative lag final long staleMillis = sequenceLastUpdated == null - ? 0 - : DateTimes.nowUtc().getMillis() - - (tuningConfig.getOffsetFetchPeriod().getMillis() + sequenceLastUpdated.getMillis()); + ? 0 + : DateTimes.nowUtc().getMillis() + - (tuningConfig.getOffsetFetchPeriod().getMillis() + + sequenceLastUpdated.getMillis()); if (staleMillis > 0 && partitionLags.values().stream().anyMatch(x -> x < 0)) { // Log at most once every twenty supervisor runs to reduce noise in the logs if ((staleMillis / getIoConfig().getPeriod().getMillis()) % 20 == 0) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java index 9ce08f936602..cdfbdbfe8554 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java @@ -62,6 +62,12 @@ public class CostBasedAutoScaler implements SupervisorTaskAutoScaler public static final String AVG_IDLE_METRIC = "task/autoScaler/costBased/pollIdleAvg"; public static final String OPTIMAL_TASK_COUNT_METRIC = "task/autoScaler/costBased/optimalTaskCount"; + enum CostComputeMode + { + NORMAL, + TASK_ROLLOVER + } + private final String supervisorId; private final SeekableStreamSupervisor supervisor; private final ServiceEmitter emitter; @@ -98,7 +104,7 @@ public CostBasedAutoScaler( @Override public void start() { - Callable scaleAction = () -> computeOptimalTaskCount(this.collectMetrics()); + Callable scaleAction = () -> computeOptimalTaskCount(this.collectMetrics(), CostComputeMode.NORMAL); Runnable onSuccessfulScale = () -> { }; @@ -129,6 +135,12 @@ public void reset() // No-op. } + @Override + public int computeTaskCountForRollover() + { + return computeOptimalTaskCount(collectMetrics(), CostComputeMode.TASK_ROLLOVER); + } + private CostMetrics collectMetrics() { if (spec.isSuspended()) { @@ -156,7 +168,7 @@ private CostMetrics collectMetrics() if (movingAvgRate > 0) { avgProcessingRate = movingAvgRate; } else { - // Fallback: estimate processing rate based on idle ratio + // Fallback: estimate processing rate based on the idle ratio final double utilizationRatio = Math.max(0.01, 1.0 - pollIdleRatio); avgProcessingRate = config.getDefaultProcessingRate() * utilizationRatio; } @@ -184,7 +196,7 @@ private CostMetrics collectMetrics() * * @return optimal task count for scale-up, or -1 if no scaling action needed */ - public int computeOptimalTaskCount(CostMetrics metrics) + public int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComputeMode) { if (metrics == null) { log.debug("No metrics available yet for supervisorId [%s]", supervisorId); @@ -243,11 +255,18 @@ public int computeOptimalTaskCount(CostMetrics metrics) metrics.getPollIdleRatio() ); + if (optimalTaskCount == currentTaskCount) { return -1; } - // Perform both scale-up and scale-down proactively - // Future versions may perform scale-down on task rollover only + // Perform scale down on task rollover only + if (optimalTaskCount < currentTaskCount) { + if (costComputeMode == CostComputeMode.TASK_ROLLOVER) { + return optimalTaskCount; + } + return -1; + } + // Scale up is performed eagerly. return optimalTaskCount; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java index 44bbef9c6a15..63d9d008d1be 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java @@ -80,15 +80,19 @@ public void testComputeValidTaskCounts() @Test public void testComputeOptimalTaskCountInvalidInputs() { - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(null)); - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(0.0, 10, 0, 0.0))); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(null, CostBasedAutoScaler.CostComputeMode.NORMAL)); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(0.0, 10, 0, 0.0), + CostBasedAutoScaler.CostComputeMode.NORMAL + )); } @Test public void testComputeOptimalTaskCountIdleInIdealRange() { // When idle is in ideal range [0.2, 0.6], no scaling should occur - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(5000.0, 25, 100, 0.4))); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(5000.0, 25, 100, 0.4), + CostBasedAutoScaler.CostComputeMode.NORMAL + )); } @Test @@ -96,7 +100,9 @@ public void testComputeOptimalTaskCountScaling() { // High idle (underutilized) - should scale down // With high idle (0.8), the algorithm evaluates lower task counts and finds they have lower idle cost - int scaleDownResult = autoScaler.computeOptimalTaskCount(createMetrics(100.0, 25, 100, 0.8)); + int scaleDownResult = autoScaler.computeOptimalTaskCount(createMetrics(100.0, 25, 100, 0.8), + CostBasedAutoScaler.CostComputeMode.NORMAL + ); Assert.assertTrue("Should scale down when idle > 0.6", scaleDownResult < 25); } @@ -110,7 +116,9 @@ public void testComputeOptimalTaskCountLowIdleDoesNotScaleUpWithBalancedWeights( // // This is intentional: the idle-heavy weights (0.4 idle) make the algorithm // favor stability over aggressive scaling - int result = autoScaler.computeOptimalTaskCount(createMetrics(1000.0, 25, 100, 0.1)); + int result = autoScaler.computeOptimalTaskCount(createMetrics(1000.0, 25, 100, 0.1), + CostBasedAutoScaler.CostComputeMode.NORMAL + ); // Algorithm evaluates costs and may find current count optimal // or may scale down if idle cost reduction outweighs lag increase diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/autoscaler/SupervisorTaskAutoScaler.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/autoscaler/SupervisorTaskAutoScaler.java index c921e2740b87..17cd347231ae 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/autoscaler/SupervisorTaskAutoScaler.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/autoscaler/SupervisorTaskAutoScaler.java @@ -24,4 +24,15 @@ public interface SupervisorTaskAutoScaler void start(); void stop(); void reset(); + + /** + * Computes the optimal task count during task rollover, allowing a non-disruptive scale-down. + * Must be called by the supervisor when tasks are ending their duration. + * + * @return optimal task count for scale-down, or -1 if no change needed + */ + default int computeTaskCountForRollover() + { + return -1; + } } From 08eddc32da5366f0cb257fab9a0dab018033b083 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Fri, 19 Dec 2025 14:49:23 +0200 Subject: [PATCH 02/20] Address feedback from the previuous PR post-review --- .../autoscaler/CostBasedAutoScaler.java | 78 ++++++++++--------- .../autoscaler/WeightedCostFunction.java | 26 ++++--- .../autoscaler/CostBasedAutoScalerTest.java | 11 +-- .../autoscaler/WeightedCostFunctionTest.java | 54 ++++++------- 4 files changed, 88 insertions(+), 81 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java index cdfbdbfe8554..8c579c0b905e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.seekablestream.supervisor.autoscaler; +import io.vavr.Tuple3; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; @@ -33,9 +34,9 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.RowIngestionMeters; -import java.util.ArrayList; -import java.util.List; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -58,8 +59,8 @@ public class CostBasedAutoScaler implements SupervisorTaskAutoScaler private static final int MAX_INCREASE_IN_PARTITIONS_PER_TASK = 2; private static final int MAX_DECREASE_IN_PARTITIONS_PER_TASK = MAX_INCREASE_IN_PARTITIONS_PER_TASK * 2; - public static final String AVG_LAG_METRIC = "task/autoScaler/costBased/avgLag"; - public static final String AVG_IDLE_METRIC = "task/autoScaler/costBased/pollIdleAvg"; + public static final String LAG_COST_METRIC = "task/autoScaler/costBased/lagCost"; + public static final String IDLE_COST_METRIC = "task/autoScaler/costBased/idleCost"; public static final String OPTIMAL_TASK_COUNT_METRIC = "task/autoScaler/costBased/optimalTaskCount"; enum CostComputeMode @@ -92,7 +93,8 @@ public CostBasedAutoScaler( this.costFunction = new WeightedCostFunction(); - this.autoscalerExecutor = Execs.scheduledSingleThreaded("CostBasedAutoScaler-" + StringUtils.encodeForFormat(spec.getId())); + this.autoscalerExecutor = Execs.scheduledSingleThreaded("CostBasedAutoScaler-" + + StringUtils.encodeForFormat(spec.getId())); this.metricBuilder = ServiceMetricEvent.builder() .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) .setDimension( @@ -158,7 +160,7 @@ private CostMetrics collectMetrics() final int partitionCount = supervisor.getPartitionCount(); final Map> taskStats = supervisor.getStats(); - final double movingAvgRate = extractMovingAverage(taskStats, DropwizardRowIngestionMeters.ONE_MINUTE_NAME); + final double movingAvgRate = extractMovingAverage(taskStats); final double pollIdleRatio = extractPollIdleRatio(taskStats); final double avgPartitionLag = lagStats.getAvgLag(); @@ -216,33 +218,30 @@ public int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComp return -1; } - // If idle is already in the ideal range [0.2, 0.6], optimal utilization has been achieved. - // No scaling is needed - maintain stability by staying at the current task count. - final double currentIdleRatio = metrics.getPollIdleRatio(); - if (currentIdleRatio >= 0 && WeightedCostFunction.isIdleInIdealRange(currentIdleRatio)) { - log.debug( - "Idle ratio [%.3f] is in ideal range for supervisorId [%s], no scaling needed", - currentIdleRatio, - supervisorId - ); - return -1; - } - int optimalTaskCount = -1; - double optimalCost = Double.POSITIVE_INFINITY; + Tuple3 optimalCost = new Tuple3<>(Double.POSITIVE_INFINITY, + Double.POSITIVE_INFINITY, + Double.POSITIVE_INFINITY); for (int taskCount : validTaskCounts) { - double cost = costFunction.computeCost(metrics, taskCount, config); - log.debug("Proposed task count: %d, Cost: %.4f", taskCount, cost); - if (cost < optimalCost) { + Tuple3 costResult = costFunction.computeCost(metrics, taskCount, config); + double cost = costResult._1(); + log.debug( + "Proposed task count: %d, Cost: %.4f (lag: %.4f, idle: %.4f)", + taskCount, + cost, + costResult._2(), + costResult._3() + ); + if (cost < optimalCost._1()) { optimalTaskCount = taskCount; - optimalCost = cost; + optimalCost = costResult; } } - emitter.emit(metricBuilder.setMetric(AVG_LAG_METRIC, metrics.getAvgPartitionLag())); - emitter.emit(metricBuilder.setMetric(AVG_IDLE_METRIC, metrics.getPollIdleRatio())); emitter.emit(metricBuilder.setMetric(OPTIMAL_TASK_COUNT_METRIC, (long) optimalTaskCount)); + emitter.emit(metricBuilder.setMetric(LAG_COST_METRIC, optimalCost._2())); + emitter.emit(metricBuilder.setMetric(IDLE_COST_METRIC, optimalCost._3())); log.debug( "Cost-based scaling evaluation for supervisorId [%s]: current=%d, optimal=%d, cost=%.4f, " @@ -250,7 +249,7 @@ public int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComp supervisorId, metrics.getCurrentTaskCount(), optimalTaskCount, - optimalCost, + optimalCost._1(), metrics.getAvgPartitionLag(), metrics.getPollIdleRatio() ); @@ -283,17 +282,18 @@ static int[] computeValidTaskCounts(int partitionCount, int currentTaskCount) return new int[]{}; } - List result = new ArrayList<>(); + Set result = new HashSet<>(); final int currentPartitionsPerTask = partitionCount / currentTaskCount; // Minimum partitions per task correspond to the maximum number of tasks (scale up) and vice versa. final int minPartitionsPerTask = Math.max(1, currentPartitionsPerTask - MAX_INCREASE_IN_PARTITIONS_PER_TASK); - final int maxPartitionsPerTask = Math.min(partitionCount, currentPartitionsPerTask + MAX_DECREASE_IN_PARTITIONS_PER_TASK); + final int maxPartitionsPerTask = Math.min( + partitionCount, + currentPartitionsPerTask + MAX_DECREASE_IN_PARTITIONS_PER_TASK + ); for (int partitionsPerTask = maxPartitionsPerTask; partitionsPerTask >= minPartitionsPerTask; partitionsPerTask--) { final int taskCount = (partitionCount + partitionsPerTask - 1) / partitionsPerTask; - if (result.isEmpty() || result.get(result.size() - 1) != taskCount) { - result.add(taskCount); - } + result.add(taskCount); } return result.stream().mapToInt(Integer::intValue).toArray(); } @@ -343,9 +343,9 @@ static double extractPollIdleRatio(Map> taskStats) * * @param taskStats the stats map from supervisor.getStats() * @return the average 15-minute processing rate across all tasks in records/second, - * or -1 if no valid metrics are available + * or -1 if no valid metrics are available */ - static double extractMovingAverage(Map> taskStats, String movingAveragePeriodKey) + static double extractMovingAverage(Map> taskStats) { if (taskStats == null || taskStats.isEmpty()) { return -1; @@ -360,9 +360,15 @@ static double extractMovingAverage(Map> taskStats, S if (movingAveragesObj instanceof Map) { Object buildSegmentsObj = ((Map) movingAveragesObj).get(RowIngestionMeters.BUILD_SEGMENTS); if (buildSegmentsObj instanceof Map) { - Object fifteenMinObj = ((Map) buildSegmentsObj).get(movingAveragePeriodKey); - if (fifteenMinObj instanceof Map) { - Object processedRate = ((Map) fifteenMinObj).get(RowIngestionMeters.PROCESSED); + Object movingAvgObj = ((Map) buildSegmentsObj).get(DropwizardRowIngestionMeters.FIFTEEN_MINUTE_NAME); + if (movingAvgObj == null) { + movingAvgObj = ((Map) buildSegmentsObj).get(DropwizardRowIngestionMeters.FIVE_MINUTE_NAME); + if (movingAvgObj == null) { + movingAvgObj = ((Map) buildSegmentsObj).get(DropwizardRowIngestionMeters.ONE_MINUTE_NAME); + } + } + if (movingAvgObj instanceof Map) { + Object processedRate = ((Map) movingAvgObj).get(RowIngestionMeters.PROCESSED); if (processedRate instanceof Number) { sum += ((Number) processedRate).doubleValue(); count++; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java index 1af8233527a5..8999b933568a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.seekablestream.supervisor.autoscaler; +import io.vavr.Tuple3; import org.apache.druid.java.util.common.logger.Logger; /** @@ -58,14 +59,15 @@ public static boolean isIdleInIdealRange(double idleRatio) * *

* Formula: {@code lagWeight * lagRecoveryTime + idleWeight * idlenessCost}. - * This approach directly connects costs to operational metricsю + * This approach directly connects costs to operational metrics. * - * @return cost score in seconds, or {@link Double#POSITIVE_INFINITY} for invalid inputs + * @return Tuple3 containing (totalCost, lagCost, idleCost), + * or tuple with {@link Double#POSITIVE_INFINITY} for invalid inputs */ - public double computeCost(CostMetrics metrics, int proposedTaskCount, CostBasedAutoScalerConfig config) + public Tuple3 computeCost(CostMetrics metrics, int proposedTaskCount, CostBasedAutoScalerConfig config) { if (metrics == null || config == null || proposedTaskCount <= 0 || metrics.getPartitionCount() <= 0) { - return Double.POSITIVE_INFINITY; + return new Tuple3<>(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY); } final double avgProcessingRate = metrics.getAvgProcessingRate(); @@ -74,9 +76,9 @@ public double computeCost(CostMetrics metrics, int proposedTaskCount, CostBasedA // Metrics are unavailable - favor maintaining the current task count. // We're conservative about scale up, but won't let an unlikey scale down to happen. if (proposedTaskCount == metrics.getCurrentTaskCount()) { - return 0.01d; + return new Tuple3<>(0.01d, 0.0, 0.0); } else { - return Double.POSITIVE_INFINITY; + return new Tuple3<>(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY); } } else { // Lag recovery time is decreasing by adding tasks and increasing by ejecting tasks. @@ -86,19 +88,21 @@ public double computeCost(CostMetrics metrics, int proposedTaskCount, CostBasedA final double predictedIdleRatio = estimateIdleRatio(metrics, proposedTaskCount); final double idleCost = proposedTaskCount * metrics.getTaskDurationSeconds() * predictedIdleRatio; - final double cost = config.getLagWeight() * lagRecoveryTime + config.getIdleWeight() * idleCost; + final double lagCost = config.getLagWeight() * lagRecoveryTime; + final double weightedIdleCost = config.getIdleWeight() * idleCost; + final double cost = lagCost + weightedIdleCost; log.debug( - "Cost for taskCount[%d]: lagRecoveryTime[%.2fs], idleCost[%.2fs], " + "Cost for taskCount[%d]: lagCost[%.2fs], idleCost[%.2fs], " + "predictedIdle[%.3f], finalCost[%.2fs]", proposedTaskCount, - lagRecoveryTime, - idleCost, + lagCost, + weightedIdleCost, predictedIdleRatio, cost ); - return cost; + return new Tuple3<>(cost, lagCost, weightedIdleCost); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java index 63d9d008d1be..19561f054cb3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java @@ -155,22 +155,19 @@ public void testExtractProcessingRateMovingAverage() // Null and empty return -1 Assert.assertEquals( -1., - CostBasedAutoScaler.extractMovingAverage(null, DropwizardRowIngestionMeters.FIVE_MINUTE_NAME), + CostBasedAutoScaler.extractMovingAverage(null), 0.0001 ); Assert.assertEquals( -1., - CostBasedAutoScaler.extractMovingAverage( - Collections.emptyMap(), - DropwizardRowIngestionMeters.FIVE_MINUTE_NAME - ), + CostBasedAutoScaler.extractMovingAverage(Collections.emptyMap()), 0.0001 ); // Missing metrics return -1 Map> missingMetrics = new HashMap<>(); missingMetrics.put("0", Collections.singletonMap("task-0", new HashMap<>())); - Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(missingMetrics, DropwizardRowIngestionMeters.FIVE_MINUTE_NAME), 0.0001); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(missingMetrics), 0.0001); // Valid stats return average Map> validStats = new HashMap<>(); @@ -178,7 +175,7 @@ public void testExtractProcessingRateMovingAverage() group.put("task-0", buildTaskStatsWithMovingAverage(1000.0)); group.put("task-1", buildTaskStatsWithMovingAverage(2000.0)); validStats.put("0", group); - Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(validStats, DropwizardRowIngestionMeters.FIVE_MINUTE_NAME), 0.0001); + Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(validStats), 0.0001); } private CostMetrics createMetrics( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java index 0c5602052d4d..1d16ba0a494b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java @@ -47,11 +47,11 @@ public void testComputeCostInvalidInputs() { CostMetrics validMetrics = createMetrics(100000.0, 10, 100, 0.3); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(null, 10, config), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 10, null), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 0, config), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, -5, config), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(createMetrics(0.0, 10, 0, 0.3), 10, config), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(null, 10, config)._1(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 10, null)._1(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 0, config)._1(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, -5, config)._1(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(createMetrics(0.0, 10, 0, 0.3), 10, config)._1(), 0.0); } @Test @@ -68,8 +68,8 @@ public void testScaleDownHasHigherLagCostThanCurrent() CostMetrics metrics = createMetrics(200000.0, 10, 200, 0.3); - double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig); - double costScaleDown = costFunction.computeCost(metrics, 5, lagOnlyConfig); + double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig)._1(); + double costScaleDown = costFunction.computeCost(metrics, 5, lagOnlyConfig)._1(); // Scale down uses absolute model: lag / (5 * rate) = higher recovery time // Current uses absolute model: lag / (10 * rate) = lower recovery time @@ -97,15 +97,15 @@ public void testLagCostWithMarginalModel() CostMetrics metrics = createMetrics(100000.0, 10, 100, 0.3); // Current (10 tasks): uses absolute model = 10M / (10 * 1000) = 1000s - double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig); + double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig)._1(); Assert.assertEquals("Cost at current tasks", 1000., costCurrent, 0.1); // Scale up by 5 (to 15): marginal model = 10M / (15 * 1000) = 666 - double costUp5 = costFunction.computeCost(metrics, 15, lagOnlyConfig); + double costUp5 = costFunction.computeCost(metrics, 15, lagOnlyConfig)._1(); Assert.assertEquals("Cost when scaling up by 5", 666.7, costUp5, 0.1); // Scale up by 10 (to 20): marginal model = 10M / (20 * 1000) = 500s - double costUp10 = costFunction.computeCost(metrics, 20, lagOnlyConfig); + double costUp10 = costFunction.computeCost(metrics, 20, lagOnlyConfig)._1(); Assert.assertEquals("Cost when scaling up by 10", 500.0, costUp10, 0.01); // Adding more tasks reduces lag recovery time @@ -120,8 +120,8 @@ public void testBalancedWeightsFavorStabilityOverScaleUp() // This is intentional behavior: the algorithm is conservative about scale-up. CostMetrics metrics = createMetrics(100000.0, 10, 100, 0.3); - double costCurrent = costFunction.computeCost(metrics, 10, config); - double costScaleUp = costFunction.computeCost(metrics, 20, config); + double costCurrent = costFunction.computeCost(metrics, 10, config)._1(); + double costScaleUp = costFunction.computeCost(metrics, 20, config)._1(); // With balanced weights (0.3 lag, 0.7 idle), the idle cost increase from // scaling up dominates the lag recovery benefit @@ -154,8 +154,8 @@ public void testWeightsAffectCost() CostMetrics metrics = createMetrics(100000.0, 10, 100, 0.1); - double costLag = costFunction.computeCost(metrics, 10, lagOnly); - double costIdle = costFunction.computeCost(metrics, 10, idleOnly); + double costLag = costFunction.computeCost(metrics, 10, lagOnly)._1(); + double costIdle = costFunction.computeCost(metrics, 10, idleOnly)._1(); Assert.assertNotEquals("Different weights should produce different costs", costLag, costIdle, 0.0001); Assert.assertTrue("Lag-only cost should be positive", costLag > 0.0); @@ -170,9 +170,9 @@ public void testNoProcessingRateFavorsCurrentTaskCount() int currentTaskCount = 10; CostMetrics metricsNoRate = createMetricsWithRate(50000.0, currentTaskCount, 100, 0.3, 0.0); - double costAtCurrent = costFunction.computeCost(metricsNoRate, currentTaskCount, config); - double costScaleUp = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, config); - double costScaleDown = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, config); + double costAtCurrent = costFunction.computeCost(metricsNoRate, currentTaskCount, config)._1(); + double costScaleUp = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, config)._1(); + double costScaleDown = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, config)._1(); Assert.assertTrue( "Cost at current should be less than cost for scale up", @@ -201,8 +201,8 @@ public void testNoProcessingRateDeviationPenaltyIsSymmetric() .defaultProcessingRate(1000.0) .build(); - double costUp5 = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, lagOnlyConfig); - double costDown5 = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, lagOnlyConfig); + double costUp5 = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, lagOnlyConfig)._1(); + double costDown5 = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, lagOnlyConfig)._1(); Assert.assertEquals( "Lag cost for +5 and -5 deviation should be equal", @@ -229,10 +229,10 @@ public void testIdleCostMonotonicWithTaskCount() // Current: 10 tasks with 40% idle (60% busy) CostMetrics metrics = createMetrics(0.0, 10, 100, 0.4); - double costAt5 = costFunction.computeCost(metrics, 5, idleOnlyConfig); - double costAt10 = costFunction.computeCost(metrics, 10, idleOnlyConfig); - double costAt15 = costFunction.computeCost(metrics, 15, idleOnlyConfig); - double costAt20 = costFunction.computeCost(metrics, 20, idleOnlyConfig); + double costAt5 = costFunction.computeCost(metrics, 5, idleOnlyConfig)._1(); + double costAt10 = costFunction.computeCost(metrics, 10, idleOnlyConfig)._1(); + double costAt15 = costFunction.computeCost(metrics, 15, idleOnlyConfig)._1(); + double costAt20 = costFunction.computeCost(metrics, 20, idleOnlyConfig)._1(); // Monotonically increasing idle cost as tasks increase Assert.assertTrue("cost(5) < cost(10)", costAt5 < costAt10); @@ -256,7 +256,7 @@ public void testIdleRatioClampingAtBoundaries() // busyFraction = 0.6, taskRatio = 0.2 // predictedIdle = 1 - 0.6/0.2 = 1 - 3 = -2 → clamped to 0 CostMetrics metrics = createMetrics(0.0, 10, 100, 0.4); - double costAt2 = costFunction.computeCost(metrics, 2, idleOnlyConfig); + double costAt2 = costFunction.computeCost(metrics, 2, idleOnlyConfig)._1(); // idlenessCost = taskCount * taskDuration * 0.0 (clamped) = 0 Assert.assertEquals("Idle cost should be 0 when predicted idle is clamped to 0", 0.0, costAt2, 0.0001); @@ -266,7 +266,7 @@ public void testIdleRatioClampingAtBoundaries() // busyFraction = 0.9, taskRatio = 10 // predictedIdle = 1 - 0.9/10 = 1 - 0.09 = 0.91 (within bounds) CostMetrics lowIdle = createMetrics(0.0, 10, 100, 0.1); - double costAt100 = costFunction.computeCost(lowIdle, 100, idleOnlyConfig); + double costAt100 = costFunction.computeCost(lowIdle, 100, idleOnlyConfig)._1(); // idlenessCost = 100 * 3600 * 0.91 = 327600 Assert.assertTrue("Cost should be finite and positive", Double.isFinite(costAt100) && costAt100 > 0); } @@ -286,8 +286,8 @@ public void testIdleRatioWithMissingData() // Negative idle ratio indicates missing data → should default to 0.5 CostMetrics missingIdleData = createMetrics(0.0, 10, 100, -1.0); - double cost10 = costFunction.computeCost(missingIdleData, 10, idleOnlyConfig); - double cost20 = costFunction.computeCost(missingIdleData, 20, idleOnlyConfig); + double cost10 = costFunction.computeCost(missingIdleData, 10, idleOnlyConfig)._1(); + double cost20 = costFunction.computeCost(missingIdleData, 20, idleOnlyConfig)._1(); // With missing data, predicted idle = 0.5 for all task counts // idlenessCost at 10 = 10 * 3600 * 0.5 = 18000 From d6894c7a962328efa477aeb24686e5a9720b6042 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Mon, 22 Dec 2025 12:13:05 +0200 Subject: [PATCH 03/20] Addressing review comments-1 --- .../supervisor/SeekableStreamSupervisor.java | 205 ++++++------------ .../autoscaler/CostBasedAutoScaler.java | 116 +++++----- .../supervisor/autoscaler/CostResult.java | 58 +++++ .../autoscaler/WeightedCostFunction.java | 15 +- .../autoscaler/WeightedCostFunctionTest.java | 54 ++--- 5 files changed, 212 insertions(+), 236 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java 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 2259ff1aa380..3a4d5e7ddad5 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 @@ -469,19 +469,15 @@ public void handle() ); return; } - log.debug( - "PendingCompletionTaskGroups is [%s] for supervisor[%s] for dataSource[%s]", pendingCompletionTaskGroups, - supervisorId, - dataSource + log.debug("PendingCompletionTaskGroups is [%s] for supervisor[%s] for dataSource[%s]", pendingCompletionTaskGroups, + supervisorId, + dataSource ); final Integer desiredTaskCount = computeDesiredTaskCount.call(); ServiceMetricEvent.Builder event = ServiceMetricEvent.builder() .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) .setDimension(DruidMetrics.DATASOURCE, dataSource) - .setDimension( - DruidMetrics.STREAM, - getIoConfig().getStream() - ); + .setDimension(DruidMetrics.STREAM, getIoConfig().getStream()); for (CopyOnWriteArrayList list : pendingCompletionTaskGroups.values()) { if (!list.isEmpty()) { log.info( @@ -601,12 +597,7 @@ private boolean changeTaskCount(int desiredActiveTaskCount) AUTOSCALER_SCALING_TIME_METRIC, scaleActionStopwatch.millisElapsed() )); - log.info( - "Changed taskCount to [%s] for supervisor[%s] for dataSource[%s].", - desiredActiveTaskCount, - supervisorId, - dataSource - ); + log.info("Changed taskCount to [%s] for supervisor[%s] for dataSource[%s].", desiredActiveTaskCount, supervisorId, dataSource); return true; } } @@ -620,20 +611,11 @@ private void changeTaskCountInIOConfig(int desiredActiveTaskCount) MetadataSupervisorManager metadataSupervisorManager = supervisorManager.get().getMetadataSupervisorManager(); metadataSupervisorManager.insert(supervisorId, spec); } else { - log.error( - "supervisorManager is null in taskMaster, skipping scale action for supervisor[%s] for dataSource[%s].", - supervisorId, - dataSource - ); + log.error("supervisorManager is null in taskMaster, skipping scale action for supervisor[%s] for dataSource[%s].", supervisorId, dataSource); } } catch (Exception e) { - log.error( - e, - "Failed to sync taskCount to MetaStorage for supervisor[%s] for dataSource[%s].", - supervisorId, - dataSource - ); + log.error(e, "Failed to sync taskCount to MetaStorage for supervisor[%s] for dataSource[%s].", supervisorId, dataSource); } } @@ -759,11 +741,7 @@ public void handle() for (Integer taskGroupId : taskGroupIds) { TaskGroup taskGroup = activelyReadingTaskGroups.getOrDefault(taskGroupId, null); if (taskGroup == null) { - log.info( - "Tried to stop task group [%d] for supervisor [%s] that wasn't actively reading.", - taskGroupId, - supervisorId - ); + log.info("Tried to stop task group [%d] for supervisor [%s] that wasn't actively reading.", taskGroupId, supervisorId); continue; } log.info("Task group [%d] for supervisor [%s] will handoff early.", taskGroupId, supervisorId); @@ -849,9 +827,8 @@ boolean isValidTaskGroup(int taskGroupId, @Nullable TaskGroup taskGroup) log.warn("Ignoring checkpoint request because taskGroup[%d] is inactive", taskGroupId); return false; } else { - throw new ISE( - "Cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId, - activelyReadingTaskGroups + throw new ISE("Cannot find taskGroup [%s] among all activelyReadingTaskGroups [%s]", taskGroupId, + activelyReadingTaskGroups ); } } @@ -1230,7 +1207,6 @@ public void reset(@Nullable final DataSourceMetadata dataSourceMetadata) * Reset offsets with provided dataSource metadata. Validates {@code resetDataSourceMetadata}, * creates a {@code ResetOffsetsNotice} with the metadata and adds it to the notice queue. The resulting stored offsets * is a union of existing checkpointed offsets and provided offsets. - * * @param resetDataSourceMetadata required datasource metadata with offsets to reset. * @throws DruidException if any metadata attribute doesn't match the supervisor's. */ @@ -1344,12 +1320,7 @@ public void tryInit() } catch (Throwable e) { stateManager.recordThrowableEvent(e); - log.makeAlert( - e, - "Supervisor[%s] for datasource[%s] failed to handle notice", - supervisorId, - dataSource - ) + log.makeAlert(e, "Supervisor[%s] for datasource[%s] failed to handle notice", supervisorId, dataSource) .addData("noticeClass", notice.getClass().getSimpleName()) .emit(); } @@ -1819,17 +1790,10 @@ public void runInternal() stateManager.recordThrowableEvent(e); if (e instanceof StreamException) { // When a StreamException is thrown, the error message is more useful than the stack trace in telling what's wrong. - log.makeAlert( - "Exception in supervisor run loop for supervisor[%s] for dataSource[%s]: [%s]", - supervisorId, dataSource, e.getMessage() - ).emit(); + log.makeAlert("Exception in supervisor run loop for supervisor[%s] for dataSource[%s]: [%s]", + supervisorId, dataSource, e.getMessage()).emit(); } else { - log.makeAlert( - e, - "Exception in supervisor run loop for supervisor[%s] for dataSource[%s]", - supervisorId, - dataSource - ).emit(); + log.makeAlert(e, "Exception in supervisor run loop for supervisor[%s] for dataSource[%s]", supervisorId, dataSource).emit(); } } finally { @@ -1901,12 +1865,7 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) if (dataSourceMetadata == null) { // Reset everything boolean result = indexerMetadataStorageCoordinator.deleteDataSourceMetadata(supervisorId); - log.info( - "Reset supervisor[%s] for dataSource[%s] - dataSource metadata entry deleted? [%s]", - supervisorId, - dataSource, - result - ); + log.info("Reset supervisor[%s] for dataSource[%s] - dataSource metadata entry deleted? [%s]", supervisorId, dataSource, result); activelyReadingTaskGroups.values() .forEach(group -> killTasksInGroup( group, @@ -1922,12 +1881,7 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) dataSourceMetadata.getClass() ); } - log.info( - "Reset supervisor[%s] for dataSource[%s] with metadata[%s]", - supervisorId, - dataSource, - dataSourceMetadata - ); + log.info("Reset supervisor[%s] for dataSource[%s] with metadata[%s]", supervisorId, dataSource, dataSourceMetadata); // Reset only the partitions in dataSourceMetadata if it has not been reset yet @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = @@ -1984,10 +1938,7 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) } else { final DataSourceMetadata newMetadata = currentMetadata.minus(resetMetadata); try { - metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata( - supervisorId, - newMetadata - ); + metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(supervisorId, newMetadata); } catch (IOException e) { log.error("Resetting DataSourceMetadata failed [%s]", e.getMessage()); @@ -2028,17 +1979,11 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata) * existing checkpointed offsets and provided offsets; any checkpointed offsets not specified in the metadata will be * preserved as-is. If checkpoints don't exist, the provided reset datasource metdadata will be inserted into * the metadata storage. Once the offsets are reset, any active tasks serving the partition offsets will be restarted. - * * @param dataSourceMetadata Required reset data source metdata. Assumed that the metadata is validated. */ public void resetOffsetsInternal(@Nonnull final DataSourceMetadata dataSourceMetadata) { - log.info( - "Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s]", - supervisorId, - dataSource, - dataSourceMetadata - ); + log.info("Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s]", supervisorId, dataSource, dataSourceMetadata); @SuppressWarnings("unchecked") final SeekableStreamDataSourceMetadata resetMetadata = @@ -2047,12 +1992,7 @@ public void resetOffsetsInternal(@Nonnull final DataSourceMetadata dataSourceMet final boolean metadataUpdateSuccess; final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(supervisorId); if (metadata == null) { - log.info( - "Checkpointed metadata in null for supervisor[%s] for dataSource[%s] - inserting metadata[%s]", - supervisorId, - dataSource, - resetMetadata - ); + log.info("Checkpointed metadata in null for supervisor[%s] for dataSource[%s] - inserting metadata[%s]", supervisorId, dataSource, resetMetadata); metadataUpdateSuccess = indexerMetadataStorageCoordinator.insertDataSourceMetadata(supervisorId, resetMetadata); } else { if (!checkSourceMetadataMatch(metadata)) { @@ -2065,35 +2005,18 @@ public void resetOffsetsInternal(@Nonnull final DataSourceMetadata dataSourceMet final SeekableStreamDataSourceMetadata currentMetadata = (SeekableStreamDataSourceMetadata) metadata; final DataSourceMetadata newMetadata = currentMetadata.plus(resetMetadata); - log.info( - "Current checkpointed metadata[%s], new metadata[%s] for supervisor[%s] for dataSource[%s]", - currentMetadata, - newMetadata, - supervisorId, - dataSource - ); + log.info("Current checkpointed metadata[%s], new metadata[%s] for supervisor[%s] for dataSource[%s]", currentMetadata, newMetadata, supervisorId, dataSource); try { metadataUpdateSuccess = indexerMetadataStorageCoordinator.resetDataSourceMetadata(supervisorId, newMetadata); } catch (IOException e) { - log.error( - "Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s] failed [%s]", - supervisorId, - dataSource, - newMetadata, - e.getMessage() - ); + log.error("Reset offsets for supervisor[%s] for dataSource[%s] with metadata[%s] failed [%s]", supervisorId, dataSource, newMetadata, e.getMessage()); throw new RuntimeException(e); } } if (!metadataUpdateSuccess) { - throw new ISE( - "Unable to reset metadata[%s] for supervisor[%s] for dataSource[%s]", - supervisorId, - dataSource, - dataSourceMetadata - ); + throw new ISE("Unable to reset metadata[%s] for supervisor[%s] for dataSource[%s]", supervisorId, dataSource, dataSourceMetadata); } resetMetadata.getSeekableStreamSequenceNumbers() @@ -2403,12 +2326,7 @@ public Boolean apply(Pair computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() - ) - ) - .forEach(entry -> { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); - final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); - if (stopTasksEarly || group.getHandoffEarly()) { - // If handoffEarly has been set, stop tasks irrespective of stopTaskCount - log.info( - "Stopping taskGroup[%d] early after running for duration[%s].", - groupId, runDuration - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - if (group.getHandoffEarly()) { - numStoppedTasks.getAndIncrement(); - } - } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { - // Stop this task group if it has run longer than the configured duration - // and the pending task groups are less than the configured stop task count. - int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() - .mapToInt(List::size).sum(); - if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { - log.info( - "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", - groupId, runDuration, ioConfig.getTaskDuration() - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - numStoppedTasks.getAndIncrement(); - } - } - }); + Comparator.comparingLong( + taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() + ) + ) + .forEach(entry -> { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); + final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); + if (stopTasksEarly || group.getHandoffEarly()) { + // If handoffEarly has been set, stop tasks irrespective of stopTaskCount + log.info( + "Stopping taskGroup[%d] early after running for duration[%s].", + groupId, runDuration + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + if (group.getHandoffEarly()) { + numStoppedTasks.getAndIncrement(); + } + } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { + // Stop this task group if it has run longer than the configured duration + // and the pending task groups are less than the configured stop task count. + int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() + .mapToInt(List::size).sum(); + if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { + log.info( + "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", + groupId, runDuration, ioConfig.getTaskDuration() + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + numStoppedTasks.getAndIncrement(); + } + } + }); List>> results = coalesceAndAwait(futures); for (int j = 0; j < results.size(); j++) { @@ -4801,8 +4719,7 @@ protected void emitLag() final long staleMillis = sequenceLastUpdated == null ? 0 : DateTimes.nowUtc().getMillis() - - (tuningConfig.getOffsetFetchPeriod().getMillis() - + sequenceLastUpdated.getMillis()); + - (tuningConfig.getOffsetFetchPeriod().getMillis() + sequenceLastUpdated.getMillis()); if (staleMillis > 0 && partitionLags.values().stream().anyMatch(x -> x < 0)) { // Log at most once every twenty supervisor runs to reduce noise in the logs if ((staleMillis / getIoConfig().getPeriod().getMillis()) % 20 == 0) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java index 8c579c0b905e..12519b65ea7c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.seekablestream.supervisor.autoscaler; -import io.vavr.Tuple3; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; @@ -143,46 +142,9 @@ public int computeTaskCountForRollover() return computeOptimalTaskCount(collectMetrics(), CostComputeMode.TASK_ROLLOVER); } - private CostMetrics collectMetrics() + public CostBasedAutoScalerConfig getConfig() { - if (spec.isSuspended()) { - log.debug("Supervisor [%s] is suspended, skipping a metrics collection", supervisorId); - return null; - } - - final LagStats lagStats = supervisor.computeLagStats(); - if (lagStats == null) { - log.debug("Lag stats unavailable for supervisorId [%s], skipping collection", supervisorId); - return null; - } - - final int currentTaskCount = supervisor.getIoConfig().getTaskCount(); - final int partitionCount = supervisor.getPartitionCount(); - - final Map> taskStats = supervisor.getStats(); - final double movingAvgRate = extractMovingAverage(taskStats); - final double pollIdleRatio = extractPollIdleRatio(taskStats); - - final double avgPartitionLag = lagStats.getAvgLag(); - - // Use an actual 15-minute moving average processing rate if available - final double avgProcessingRate; - if (movingAvgRate > 0) { - avgProcessingRate = movingAvgRate; - } else { - // Fallback: estimate processing rate based on the idle ratio - final double utilizationRatio = Math.max(0.01, 1.0 - pollIdleRatio); - avgProcessingRate = config.getDefaultProcessingRate() * utilizationRatio; - } - - return new CostMetrics( - avgPartitionLag, - currentTaskCount, - partitionCount, - pollIdleRatio, - supervisor.getIoConfig().getTaskDuration().getStandardSeconds(), - avgProcessingRate - ); + return config; } /** @@ -198,7 +160,7 @@ private CostMetrics collectMetrics() * * @return optimal task count for scale-up, or -1 if no scaling action needed */ - public int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComputeMode) + int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComputeMode) { if (metrics == null) { log.debug("No metrics available yet for supervisorId [%s]", supervisorId); @@ -219,29 +181,31 @@ public int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComp } int optimalTaskCount = -1; - Tuple3 optimalCost = new Tuple3<>(Double.POSITIVE_INFINITY, - Double.POSITIVE_INFINITY, - Double.POSITIVE_INFINITY); + CostResult optimalCost = new CostResult( + Double.POSITIVE_INFINITY, + Double.POSITIVE_INFINITY, + Double.POSITIVE_INFINITY + ); for (int taskCount : validTaskCounts) { - Tuple3 costResult = costFunction.computeCost(metrics, taskCount, config); - double cost = costResult._1(); + CostResult costResult = costFunction.computeCost(metrics, taskCount, config); + double cost = costResult.totalCost(); log.debug( "Proposed task count: %d, Cost: %.4f (lag: %.4f, idle: %.4f)", taskCount, cost, - costResult._2(), - costResult._3() + costResult.lagCost(), + costResult.idleCost() ); - if (cost < optimalCost._1()) { + if (cost < optimalCost.totalCost()) { optimalTaskCount = taskCount; optimalCost = costResult; } } emitter.emit(metricBuilder.setMetric(OPTIMAL_TASK_COUNT_METRIC, (long) optimalTaskCount)); - emitter.emit(metricBuilder.setMetric(LAG_COST_METRIC, optimalCost._2())); - emitter.emit(metricBuilder.setMetric(IDLE_COST_METRIC, optimalCost._3())); + emitter.emit(metricBuilder.setMetric(LAG_COST_METRIC, optimalCost.lagCost())); + emitter.emit(metricBuilder.setMetric(IDLE_COST_METRIC, optimalCost.idleCost())); log.debug( "Cost-based scaling evaluation for supervisorId [%s]: current=%d, optimal=%d, cost=%.4f, " @@ -249,7 +213,7 @@ public int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComp supervisorId, metrics.getCurrentTaskCount(), optimalTaskCount, - optimalCost._1(), + optimalCost.totalCost(), metrics.getAvgPartitionLag(), metrics.getPollIdleRatio() ); @@ -298,11 +262,6 @@ static int[] computeValidTaskCounts(int partitionCount, int currentTaskCount) return result.stream().mapToInt(Integer::intValue).toArray(); } - public CostBasedAutoScalerConfig getConfig() - { - return config; - } - /** * Extracts the average poll-idle-ratio metric from task stats. * This metric indicates how much time the consumer spends idle waiting for data. @@ -382,4 +341,47 @@ static double extractMovingAverage(Map> taskStats) return count > 0 ? sum / count : -1; } + + private CostMetrics collectMetrics() + { + if (spec.isSuspended()) { + log.debug("Supervisor [%s] is suspended, skipping a metrics collection", supervisorId); + return null; + } + + final LagStats lagStats = supervisor.computeLagStats(); + if (lagStats == null) { + log.debug("Lag stats unavailable for supervisorId [%s], skipping collection", supervisorId); + return null; + } + + final int currentTaskCount = supervisor.getIoConfig().getTaskCount(); + final int partitionCount = supervisor.getPartitionCount(); + + final Map> taskStats = supervisor.getStats(); + final double movingAvgRate = extractMovingAverage(taskStats); + final double pollIdleRatio = extractPollIdleRatio(taskStats); + + final double avgPartitionLag = lagStats.getAvgLag(); + + // Use an actual 15-minute moving average processing rate if available + final double avgProcessingRate; + if (movingAvgRate > 0) { + avgProcessingRate = movingAvgRate; + } else { + // Fallback: estimate processing rate based on the idle ratio + final double utilizationRatio = Math.max(0.01, 1.0 - pollIdleRatio); + avgProcessingRate = config.getDefaultProcessingRate() * utilizationRatio; + } + + return new CostMetrics( + avgPartitionLag, + currentTaskCount, + partitionCount, + pollIdleRatio, + supervisor.getIoConfig().getTaskDuration().getStandardSeconds(), + avgProcessingRate + ); + } + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java new file mode 100644 index 000000000000..1ad946ff5175 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.seekablestream.supervisor.autoscaler; + +/** + * Holds the result of a cost computation from {@link WeightedCostFunction#computeCost}. + * All costs are measured in seconds. + */ +public class CostResult +{ + private final double totalCost; + private final double lagCost; + private final double idleCost; + + /** + * @param totalCost the weighted sum of lagCost and idleCost + * @param lagCost the weighted cost representing expected time (seconds) to recover current lag + * @param idleCost the weighted cost representing total compute time (seconds) wasted being idle per task duration + */ + public CostResult(double totalCost, double lagCost, double idleCost) + { + this.totalCost = totalCost; + this.lagCost = lagCost; + this.idleCost = idleCost; + } + + public double totalCost() + { + return totalCost; + } + + public double lagCost() + { + return lagCost; + } + + public double idleCost() + { + return idleCost; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java index 8999b933568a..0da733ef9e71 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.seekablestream.supervisor.autoscaler; -import io.vavr.Tuple3; import org.apache.druid.java.util.common.logger.Logger; /** @@ -61,13 +60,13 @@ public static boolean isIdleInIdealRange(double idleRatio) * Formula: {@code lagWeight * lagRecoveryTime + idleWeight * idlenessCost}. * This approach directly connects costs to operational metrics. * - * @return Tuple3 containing (totalCost, lagCost, idleCost), - * or tuple with {@link Double#POSITIVE_INFINITY} for invalid inputs + * @return CostResult containing totalCost, lagCost, and idleCost, + * or result with {@link Double#POSITIVE_INFINITY} for invalid inputs */ - public Tuple3 computeCost(CostMetrics metrics, int proposedTaskCount, CostBasedAutoScalerConfig config) + public CostResult computeCost(CostMetrics metrics, int proposedTaskCount, CostBasedAutoScalerConfig config) { if (metrics == null || config == null || proposedTaskCount <= 0 || metrics.getPartitionCount() <= 0) { - return new Tuple3<>(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY); + return new CostResult(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY); } final double avgProcessingRate = metrics.getAvgProcessingRate(); @@ -76,9 +75,9 @@ public Tuple3 computeCost(CostMetrics metrics, int propo // Metrics are unavailable - favor maintaining the current task count. // We're conservative about scale up, but won't let an unlikey scale down to happen. if (proposedTaskCount == metrics.getCurrentTaskCount()) { - return new Tuple3<>(0.01d, 0.0, 0.0); + return new CostResult(0.01d, 0.0, 0.0); } else { - return new Tuple3<>(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY); + return new CostResult(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY); } } else { // Lag recovery time is decreasing by adding tasks and increasing by ejecting tasks. @@ -102,7 +101,7 @@ public Tuple3 computeCost(CostMetrics metrics, int propo cost ); - return new Tuple3<>(cost, lagCost, weightedIdleCost); + return new CostResult(cost, lagCost, weightedIdleCost); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java index 1d16ba0a494b..90b50477c924 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunctionTest.java @@ -47,11 +47,11 @@ public void testComputeCostInvalidInputs() { CostMetrics validMetrics = createMetrics(100000.0, 10, 100, 0.3); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(null, 10, config)._1(), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 10, null)._1(), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 0, config)._1(), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, -5, config)._1(), 0.0); - Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(createMetrics(0.0, 10, 0, 0.3), 10, config)._1(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(null, 10, config).totalCost(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 10, null).totalCost(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, 0, config).totalCost(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(validMetrics, -5, config).totalCost(), 0.0); + Assert.assertEquals(Double.POSITIVE_INFINITY, costFunction.computeCost(createMetrics(0.0, 10, 0, 0.3), 10, config).totalCost(), 0.0); } @Test @@ -68,8 +68,8 @@ public void testScaleDownHasHigherLagCostThanCurrent() CostMetrics metrics = createMetrics(200000.0, 10, 200, 0.3); - double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig)._1(); - double costScaleDown = costFunction.computeCost(metrics, 5, lagOnlyConfig)._1(); + double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig).totalCost(); + double costScaleDown = costFunction.computeCost(metrics, 5, lagOnlyConfig).totalCost(); // Scale down uses absolute model: lag / (5 * rate) = higher recovery time // Current uses absolute model: lag / (10 * rate) = lower recovery time @@ -97,15 +97,15 @@ public void testLagCostWithMarginalModel() CostMetrics metrics = createMetrics(100000.0, 10, 100, 0.3); // Current (10 tasks): uses absolute model = 10M / (10 * 1000) = 1000s - double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig)._1(); + double costCurrent = costFunction.computeCost(metrics, 10, lagOnlyConfig).totalCost(); Assert.assertEquals("Cost at current tasks", 1000., costCurrent, 0.1); // Scale up by 5 (to 15): marginal model = 10M / (15 * 1000) = 666 - double costUp5 = costFunction.computeCost(metrics, 15, lagOnlyConfig)._1(); + double costUp5 = costFunction.computeCost(metrics, 15, lagOnlyConfig).totalCost(); Assert.assertEquals("Cost when scaling up by 5", 666.7, costUp5, 0.1); // Scale up by 10 (to 20): marginal model = 10M / (20 * 1000) = 500s - double costUp10 = costFunction.computeCost(metrics, 20, lagOnlyConfig)._1(); + double costUp10 = costFunction.computeCost(metrics, 20, lagOnlyConfig).totalCost(); Assert.assertEquals("Cost when scaling up by 10", 500.0, costUp10, 0.01); // Adding more tasks reduces lag recovery time @@ -120,8 +120,8 @@ public void testBalancedWeightsFavorStabilityOverScaleUp() // This is intentional behavior: the algorithm is conservative about scale-up. CostMetrics metrics = createMetrics(100000.0, 10, 100, 0.3); - double costCurrent = costFunction.computeCost(metrics, 10, config)._1(); - double costScaleUp = costFunction.computeCost(metrics, 20, config)._1(); + double costCurrent = costFunction.computeCost(metrics, 10, config).totalCost(); + double costScaleUp = costFunction.computeCost(metrics, 20, config).totalCost(); // With balanced weights (0.3 lag, 0.7 idle), the idle cost increase from // scaling up dominates the lag recovery benefit @@ -154,8 +154,8 @@ public void testWeightsAffectCost() CostMetrics metrics = createMetrics(100000.0, 10, 100, 0.1); - double costLag = costFunction.computeCost(metrics, 10, lagOnly)._1(); - double costIdle = costFunction.computeCost(metrics, 10, idleOnly)._1(); + double costLag = costFunction.computeCost(metrics, 10, lagOnly).totalCost(); + double costIdle = costFunction.computeCost(metrics, 10, idleOnly).totalCost(); Assert.assertNotEquals("Different weights should produce different costs", costLag, costIdle, 0.0001); Assert.assertTrue("Lag-only cost should be positive", costLag > 0.0); @@ -170,9 +170,9 @@ public void testNoProcessingRateFavorsCurrentTaskCount() int currentTaskCount = 10; CostMetrics metricsNoRate = createMetricsWithRate(50000.0, currentTaskCount, 100, 0.3, 0.0); - double costAtCurrent = costFunction.computeCost(metricsNoRate, currentTaskCount, config)._1(); - double costScaleUp = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, config)._1(); - double costScaleDown = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, config)._1(); + double costAtCurrent = costFunction.computeCost(metricsNoRate, currentTaskCount, config).totalCost(); + double costScaleUp = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, config).totalCost(); + double costScaleDown = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, config).totalCost(); Assert.assertTrue( "Cost at current should be less than cost for scale up", @@ -201,8 +201,8 @@ public void testNoProcessingRateDeviationPenaltyIsSymmetric() .defaultProcessingRate(1000.0) .build(); - double costUp5 = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, lagOnlyConfig)._1(); - double costDown5 = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, lagOnlyConfig)._1(); + double costUp5 = costFunction.computeCost(metricsNoRate, currentTaskCount + 5, lagOnlyConfig).totalCost(); + double costDown5 = costFunction.computeCost(metricsNoRate, currentTaskCount - 5, lagOnlyConfig).totalCost(); Assert.assertEquals( "Lag cost for +5 and -5 deviation should be equal", @@ -229,10 +229,10 @@ public void testIdleCostMonotonicWithTaskCount() // Current: 10 tasks with 40% idle (60% busy) CostMetrics metrics = createMetrics(0.0, 10, 100, 0.4); - double costAt5 = costFunction.computeCost(metrics, 5, idleOnlyConfig)._1(); - double costAt10 = costFunction.computeCost(metrics, 10, idleOnlyConfig)._1(); - double costAt15 = costFunction.computeCost(metrics, 15, idleOnlyConfig)._1(); - double costAt20 = costFunction.computeCost(metrics, 20, idleOnlyConfig)._1(); + double costAt5 = costFunction.computeCost(metrics, 5, idleOnlyConfig).totalCost(); + double costAt10 = costFunction.computeCost(metrics, 10, idleOnlyConfig).totalCost(); + double costAt15 = costFunction.computeCost(metrics, 15, idleOnlyConfig).totalCost(); + double costAt20 = costFunction.computeCost(metrics, 20, idleOnlyConfig).totalCost(); // Monotonically increasing idle cost as tasks increase Assert.assertTrue("cost(5) < cost(10)", costAt5 < costAt10); @@ -256,7 +256,7 @@ public void testIdleRatioClampingAtBoundaries() // busyFraction = 0.6, taskRatio = 0.2 // predictedIdle = 1 - 0.6/0.2 = 1 - 3 = -2 → clamped to 0 CostMetrics metrics = createMetrics(0.0, 10, 100, 0.4); - double costAt2 = costFunction.computeCost(metrics, 2, idleOnlyConfig)._1(); + double costAt2 = costFunction.computeCost(metrics, 2, idleOnlyConfig).totalCost(); // idlenessCost = taskCount * taskDuration * 0.0 (clamped) = 0 Assert.assertEquals("Idle cost should be 0 when predicted idle is clamped to 0", 0.0, costAt2, 0.0001); @@ -266,7 +266,7 @@ public void testIdleRatioClampingAtBoundaries() // busyFraction = 0.9, taskRatio = 10 // predictedIdle = 1 - 0.9/10 = 1 - 0.09 = 0.91 (within bounds) CostMetrics lowIdle = createMetrics(0.0, 10, 100, 0.1); - double costAt100 = costFunction.computeCost(lowIdle, 100, idleOnlyConfig)._1(); + double costAt100 = costFunction.computeCost(lowIdle, 100, idleOnlyConfig).totalCost(); // idlenessCost = 100 * 3600 * 0.91 = 327600 Assert.assertTrue("Cost should be finite and positive", Double.isFinite(costAt100) && costAt100 > 0); } @@ -286,8 +286,8 @@ public void testIdleRatioWithMissingData() // Negative idle ratio indicates missing data → should default to 0.5 CostMetrics missingIdleData = createMetrics(0.0, 10, 100, -1.0); - double cost10 = costFunction.computeCost(missingIdleData, 10, idleOnlyConfig)._1(); - double cost20 = costFunction.computeCost(missingIdleData, 20, idleOnlyConfig)._1(); + double cost10 = costFunction.computeCost(missingIdleData, 10, idleOnlyConfig).totalCost(); + double cost20 = costFunction.computeCost(missingIdleData, 20, idleOnlyConfig).totalCost(); // With missing data, predicted idle = 0.5 for all task counts // idlenessCost at 10 = 10 * 3600 * 0.5 = 18000 From 2d8ba11c2d572015cd37da436363b6d8678a358f Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Mon, 22 Dec 2025 13:37:00 +0200 Subject: [PATCH 04/20] Use the existing method to cleanup allocation info to force partition re-assignment during the rollover --- .../seekablestream/supervisor/SeekableStreamSupervisor.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 3a4d5e7ddad5..94655d77f89a 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 @@ -3443,9 +3443,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException changeTaskCountInIOConfig(rolloverTaskCount); // Here force reset the supervisor state to be re-calculated on the next iteration of runInternal() call. // This seems the best way to inject task amount recalculation during the rollover. - partitionGroups.clear(); - partitionOffsets.clear(); - partitionIds.clear(); + clearAllocationInfo(); } } } From 7a853ab4d22f3427f4e8241f563d3e9deee6b400 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Mon, 22 Dec 2025 15:50:30 +0200 Subject: [PATCH 05/20] Move the test from supervisor suite to autoscaler it test --- .../indexing/TaskScaleDownRolloverTest.java | 232 ------------------ .../CostBasedAutoScalerIntegrationTest.java | 86 ++++++- 2 files changed, 76 insertions(+), 242 deletions(-) delete mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java deleted file mode 100644 index b29a5c99f2bd..000000000000 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/TaskScaleDownRolloverTest.java +++ /dev/null @@ -1,232 +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.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.seekablestream.supervisor.autoscaler.CostBasedAutoScalerConfig; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.query.DruidMetrics; -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.apache.kafka.clients.producer.ProducerRecord; -import org.hamcrest.Matchers; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Seconds; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; - -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import static org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler.OPTIMAL_TASK_COUNT_METRIC; - -@SuppressWarnings("SameParameterValue") -public class TaskScaleDownRolloverTest extends EmbeddedClusterTestBase -{ - private static final String TOPIC = EmbeddedClusterApis.createTestDatasourceName(); - private static final String EVENT_TEMPLATE = "{\"timestamp\":\"%s\",\"dimension\":\"value%d\",\"metric\":%d}"; - private static final int PARTITION_COUNT = 50; - - private final EmbeddedIndexer indexer = new EmbeddedIndexer(); - private final EmbeddedOverlord overlord = new EmbeddedOverlord(); - private KafkaResource kafkaServer; - - @Override - public EmbeddedDruidCluster createCluster() - { - final EmbeddedDruidCluster cluster = EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper(); - - kafkaServer = new KafkaResource() - { - @Override - public void start() - { - super.start(); - createTopicWithPartitions(TOPIC, PARTITION_COUNT); - produceRecordsToKafka(500, 1); - } - - @Override - public void stop() - { - deleteTopic(TOPIC); - super.stop(); - } - }; - - // Increase worker capacity to handle more tasks - indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") - .addProperty("druid.worker.capacity", "60"); - - overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") - .addProperty("druid.manager.segments.pollDuration", "PT0.1s"); - - cluster.useLatchableEmitter() - .useDefaultTimeoutForLatchableEmitter(20) - .addServer(new EmbeddedCoordinator()) - .addServer(overlord) - .addServer(indexer) - .addServer(new EmbeddedRouter()) - .addServer(new EmbeddedBroker()) - .addServer(new EmbeddedHistorical()) - .addExtension(KafkaIndexTaskModule.class) - .addCommonProperty("druid.monitoring.emissionPeriod", "PT1s") - .addResource(kafkaServer); - - return cluster; - } - - /** - * Tests that scale down happen during task rollover via checkTaskDuration(). - * - *

Test flow:

- *
    - *
  1. Start supervisor with 20 tasks and 50 partitions, minimal data (500 records)
  2. - *
  3. Wait for initial tasks to start running
  4. - *
  5. Wait for the first task rollover to complete (task duration is 10 seconds)
  6. - *
  7. Verify that after rollover, fewer tasks are running due to cost-based autoscaler (no ingestion at all)
  8. - *
- * - *

Scale down during rollover is triggered in {@code SeekableStreamSupervisor.checkTaskDuration()} - * when all task groups have rolled over and the autoscaler recommends a lower task count.

- */ - @SuppressWarnings("resource") - @Test - @Timeout(300) - void test_scaleDownDuringTaskRollover() - { - final String superId = dataSource + "_super"; - final int initialTaskCount = 10; - - final CostBasedAutoScalerConfig autoScalerConfig = CostBasedAutoScalerConfig - .builder() - .enableTaskAutoScaler(true) - .taskCountMin(1) - .taskCountMax(10) - .taskCountStart(initialTaskCount) - .scaleActionPeriodMillis(2000) - .minTriggerScaleActionFrequencyMillis(2000) - // High idle weight ensures scale-down when tasks are mostly idle (little data to process) - .lagWeight(0.1) - .idleWeight(0.9) - .build(); - - final KafkaSupervisorSpec spec = createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig, initialTaskCount); - - // Submit the supervisor - Assertions.assertEquals(superId, cluster.callApi().postSupervisor(spec)); - - // Wait for at least one task running for the datasource managed by the supervisor. - overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/run/time") - .hasDimension(DruidMetrics.DATASOURCE, dataSource)); - - // Wait for autoscaler to emit metric indicating scale-down, it should be just less than the current task count. - overlord.latchableEmitter().waitForEvent( - event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC) - .hasValueMatching(Matchers.lessThan((long) initialTaskCount))); - - // Wait for tasks to complete (first rollover) - overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/action/success/count")); - - // Wait for the task running for the datasource managed by a supervisor. - overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/run/time") - .hasDimension(DruidMetrics.DATASOURCE, dataSource)); - - // After rollover, verify that the running task count has decreased - // The autoscaler should have recommended fewer tasks due to high idle time - final int postRolloverRunningTasks = cluster.callApi().getTaskCount("running", dataSource); - - Assertions.assertTrue( - postRolloverRunningTasks < initialTaskCount, - StringUtils.format( - "Expected running task count to decrease after rollover. Initial: %d, After rollover: %d", - initialTaskCount, - postRolloverRunningTasks - ) - ); - - // Suspend the supervisor to clean up - cluster.callApi().postSupervisor(spec.createSuspendedSpec()); - } - - private KafkaSupervisorSpec createKafkaSupervisorWithAutoScaler( - String supervisorId, - CostBasedAutoScalerConfig autoScalerConfig, - int taskCount - ) - { - return MoreResources.Supervisor.KAFKA_JSON - .get() - .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) - .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(100)) - .withIoConfig( - ioConfig -> ioConfig - .withConsumerProperties(kafkaServer.consumerProperties()) - .withTaskCount(taskCount) - .withTaskDuration(Seconds.parseSeconds("PT10S").toPeriod()) - .withAutoScalerConfig(autoScalerConfig) - ) - .withId(supervisorId) - .build(dataSource, TOPIC); - } - - private void produceRecordsToKafka(int recordCount, int iterations) - { - int recordCountPerSlice = recordCount / iterations; - int counter = 0; - for (int i = 0; i < iterations; i++) { - DateTime timestamp = DateTime.now(DateTimeZone.UTC); - List> records = IntStream - .range(counter, counter + recordCountPerSlice) - .mapToObj(k -> new ProducerRecord( - TOPIC, - k % PARTITION_COUNT, - null, - StringUtils.format(EVENT_TEMPLATE, timestamp, k, k) - .getBytes(StandardCharsets.UTF_8) - ) - ) - .collect(Collectors.toList()); - - kafkaServer.produceRecordsToTopic(records); - try { - Thread.sleep(100L); - counter += recordCountPerSlice; - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - } -} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java index c5b86688ea1c..8ca35fcda87d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java @@ -59,6 +59,7 @@ *

* Tests the autoscaler's ability to compute optimal task counts based on partition count and cost metrics (lag and idle time). */ +@SuppressWarnings("resource") public class CostBasedAutoScalerIntegrationTest extends EmbeddedClusterTestBase { private static final String TOPIC = EmbeddedClusterApis.createTestDatasourceName(); @@ -95,18 +96,11 @@ public void stop() } }; - // Increase worker capacity to handle more tasks indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") - .addProperty("druid.worker.capacity", "60"); - - overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") - .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") - .addProperty("druid.manager.segments.pollDuration", "PT0.1s"); - - coordinator.addProperty("druid.manager.segments.useIncrementalCache", "ifSynced"); + .addProperty("druid.worker.capacity", "100"); cluster.useLatchableEmitter() - .useDefaultTimeoutForLatchableEmitter(120) + .useDefaultTimeoutForLatchableEmitter(60) .addServer(coordinator) .addServer(overlord) .addServer(indexer) @@ -161,6 +155,20 @@ public void test_autoScaler_computesOptimalTaskCountAndProduceScaleDown() cluster.callApi().postSupervisor(spec.createSuspendedSpec()); } + /** + * Tests that scale down happen during task rollover via checkTaskDuration(). + * + *

Test flow:

+ *
    + *
  1. Start supervisor with 20 tasks and 50 partitions, minimal data (500 records)
  2. + *
  3. Wait for initial tasks to start running
  4. + *
  5. Wait for the first task rollover to complete (task duration is 10 seconds)
  6. + *
  7. Verify that after rollover, fewer tasks are running due to cost-based autoscaler (no ingestion at all)
  8. + *
+ * + *

Scale down during rollover is triggered in {@code SeekableStreamSupervisor.checkTaskDuration()} + * when all task groups have rolled over and the autoscaler recommends a lower task count.

+ */ @Test @Timeout(125) public void test_autoScaler_computesOptimalTaskCountAndProducesScaleUp() @@ -215,6 +223,64 @@ public void test_autoScaler_computesOptimalTaskCountAndProducesScaleUp() cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); } + @Test + @Timeout(300) + void test_scaleDownDuringTaskRollover() + { + final String superId = dataSource + "_super"; + final int initialTaskCount = 10; + + final CostBasedAutoScalerConfig autoScalerConfig = CostBasedAutoScalerConfig + .builder() + .enableTaskAutoScaler(true) + .taskCountMin(1) + .taskCountMax(10) + .taskCountStart(initialTaskCount) + .scaleActionPeriodMillis(2000) + .minTriggerScaleActionFrequencyMillis(2000) + // High idle weight ensures scale-down when tasks are mostly idle (little data to process) + .lagWeight(0.1) + .idleWeight(0.9) + .build(); + + final KafkaSupervisorSpec spec = createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig, initialTaskCount); + + // Submit the supervisor + Assertions.assertEquals(superId, cluster.callApi().postSupervisor(spec)); + + // Wait for at least one task running for the datasource managed by the supervisor. + overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/run/time") + .hasDimension(DruidMetrics.DATASOURCE, dataSource)); + + // Wait for autoscaler to emit metric indicating scale-down, it should be just less than the current task count. + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC) + .hasValueMatching(Matchers.lessThan((long) initialTaskCount))); + + // Wait for tasks to complete (first rollover) + overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/action/success/count")); + + // Wait for the task running for the datasource managed by a supervisor. + overlord.latchableEmitter().waitForEvent(e -> e.hasMetricName("task/run/time") + .hasDimension(DruidMetrics.DATASOURCE, dataSource)); + + // After rollover, verify that the running task count has decreased + // The autoscaler should have recommended fewer tasks due to high idle time + final int postRolloverRunningTasks = cluster.callApi().getTaskCount("running", dataSource); + + Assertions.assertTrue( + postRolloverRunningTasks < initialTaskCount, + StringUtils.format( + "Expected running task count to decrease after rollover. Initial: %d, After rollover: %d", + initialTaskCount, + postRolloverRunningTasks + ) + ); + + // Suspend the supervisor to clean up + cluster.callApi().postSupervisor(spec.createSuspendedSpec()); + } + private void produceRecordsToKafka(int recordCount, int iterations) { int recordCountPerSlice = recordCount / iterations; @@ -258,7 +324,7 @@ private KafkaSupervisorSpec createKafkaSupervisorWithAutoScaler( ioConfig -> ioConfig .withConsumerProperties(kafkaServer.consumerProperties()) .withTaskCount(taskCount) - .withTaskDuration(Seconds.THREE.toPeriod()) + .withTaskDuration(Seconds.parseSeconds("PT7S").toPeriod()) .withAutoScalerConfig(autoScalerConfig) ) .withId(supervisorId) From 1109f5ba883e953b9840062652d88ff02aeaddbf Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Mon, 22 Dec 2025 17:11:46 +0200 Subject: [PATCH 06/20] Update test case for task rollover duration --- .../autoscaler/CostBasedAutoScalerIntegrationTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java index 8ca35fcda87d..01260f0c9462 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java @@ -160,9 +160,9 @@ public void test_autoScaler_computesOptimalTaskCountAndProduceScaleDown() * *

Test flow:

*
    - *
  1. Start supervisor with 20 tasks and 50 partitions, minimal data (500 records)
  2. + *
  3. Start supervisor with 10 tasks and 50 partitions, minimal data (500 records)
  4. *
  5. Wait for initial tasks to start running
  6. - *
  7. Wait for the first task rollover to complete (task duration is 10 seconds)
  8. + *
  9. Wait for the first task rollover to complete (task duration is 8 seconds)
  10. *
  11. Verify that after rollover, fewer tasks are running due to cost-based autoscaler (no ingestion at all)
  12. *
* From 004b10dfcd614ae2faab2b20217ad681f551f4ea Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 14:08:04 +0200 Subject: [PATCH 07/20] Formatting --- .../supervisor/SeekableStreamSupervisor.java | 78 +++++++++---------- 1 file changed, 39 insertions(+), 39 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 94655d77f89a..023b4969e338 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 @@ -1791,7 +1791,7 @@ public void runInternal() if (e instanceof StreamException) { // When a StreamException is thrown, the error message is more useful than the stack trace in telling what's wrong. log.makeAlert("Exception in supervisor run loop for supervisor[%s] for dataSource[%s]: [%s]", - supervisorId, dataSource, e.getMessage()).emit(); + supervisorId, dataSource, e.getMessage()).emit(); } else { log.makeAlert(e, "Exception in supervisor run loop for supervisor[%s] for dataSource[%s]", supervisorId, dataSource).emit(); } @@ -3345,45 +3345,45 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException final AtomicInteger numStoppedTasks = new AtomicInteger(); // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing + // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing activelyReadingTaskGroups.entrySet().stream().sorted( - Comparator.comparingLong( - taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() - ) - ) - .forEach(entry -> { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); - final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); - if (stopTasksEarly || group.getHandoffEarly()) { - // If handoffEarly has been set, stop tasks irrespective of stopTaskCount - log.info( - "Stopping taskGroup[%d] early after running for duration[%s].", - groupId, runDuration - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - if (group.getHandoffEarly()) { - numStoppedTasks.getAndIncrement(); - } - } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { - // Stop this task group if it has run longer than the configured duration - // and the pending task groups are less than the configured stop task count. - int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() - .mapToInt(List::size).sum(); - if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { - log.info( - "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", - groupId, runDuration, ioConfig.getTaskDuration() - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - numStoppedTasks.getAndIncrement(); - } - } - }); - + Comparator.comparingLong( + taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() + ) + ) + .forEach(entry -> { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); + final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); + if (stopTasksEarly || group.getHandoffEarly()) { + // If handoffEarly has been set, stop tasks irrespective of stopTaskCount + log.info( + "Stopping taskGroup[%d] early after running for duration[%s].", + groupId, runDuration + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + if (group.getHandoffEarly()) { + numStoppedTasks.getAndIncrement(); + } + } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { + // Stop this task group if it has run longer than the configured duration + // and the pending task groups are less than the configured stop task count. + int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() + .mapToInt(List::size).sum(); + if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { + log.info( + "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", + groupId, runDuration, ioConfig.getTaskDuration() + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + numStoppedTasks.getAndIncrement(); + } + } + }); List>> results = coalesceAndAwait(futures); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); From 6ce6562483e71bcc598fb0a1c5258428f6e6963d Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 14:56:53 +0200 Subject: [PATCH 08/20] Address review comments - 2 --- .../CostBasedAutoScalerIntegrationTest.java | 20 +-------- .../supervisor/SupervisorManager.java | 7 +-- .../supervisor/SeekableStreamSupervisor.java | 7 +++ .../autoscaler/CostBasedAutoScaler.java | 41 +++++++---------- .../supervisor/autoscaler/CostResult.java | 6 +++ .../autoscaler/CostBasedAutoScalerTest.java | 44 +++++-------------- .../overlord/supervisor/Supervisor.java | 6 +++ 7 files changed, 48 insertions(+), 83 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java index 01260f0c9462..140c4626d220 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java @@ -41,7 +41,7 @@ import org.hamcrest.Matchers; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import org.joda.time.Seconds; +import org.joda.time.Period; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -155,22 +155,7 @@ public void test_autoScaler_computesOptimalTaskCountAndProduceScaleDown() cluster.callApi().postSupervisor(spec.createSuspendedSpec()); } - /** - * Tests that scale down happen during task rollover via checkTaskDuration(). - * - *

Test flow:

- *
    - *
  1. Start supervisor with 10 tasks and 50 partitions, minimal data (500 records)
  2. - *
  3. Wait for initial tasks to start running
  4. - *
  5. Wait for the first task rollover to complete (task duration is 8 seconds)
  6. - *
  7. Verify that after rollover, fewer tasks are running due to cost-based autoscaler (no ingestion at all)
  8. - *
- * - *

Scale down during rollover is triggered in {@code SeekableStreamSupervisor.checkTaskDuration()} - * when all task groups have rolled over and the autoscaler recommends a lower task count.

- */ @Test - @Timeout(125) public void test_autoScaler_computesOptimalTaskCountAndProducesScaleUp() { final String superId = dataSource + "_super_scaleup"; @@ -224,7 +209,6 @@ public void test_autoScaler_computesOptimalTaskCountAndProducesScaleUp() } @Test - @Timeout(300) void test_scaleDownDuringTaskRollover() { final String superId = dataSource + "_super"; @@ -324,7 +308,7 @@ private KafkaSupervisorSpec createKafkaSupervisorWithAutoScaler( ioConfig -> ioConfig .withConsumerProperties(kafkaServer.consumerProperties()) .withTaskCount(taskCount) - .withTaskDuration(Seconds.parseSeconds("PT7S").toPeriod()) + .withTaskDuration(Period.seconds(7)) .withAutoScalerConfig(autoScalerConfig) ) .withId(supervisorId) 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 ccc31ef639e4..f0e959fe827d 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 @@ -502,12 +502,7 @@ private boolean createAndStartSupervisorInternal(SupervisorSpec spec, boolean pe SupervisorTaskAutoScaler autoscaler; try { supervisor = spec.createSupervisor(); - autoscaler = spec.createAutoscaler(supervisor); - - // Wire autoscaler back to supervisor for rollover-based scale-down - if (supervisor instanceof SeekableStreamSupervisor && autoscaler != null) { - ((SeekableStreamSupervisor) supervisor).setTaskAutoScaler(autoscaler); - } + autoscaler = supervisor.createAutoscaler(); supervisor.start(); if (autoscaler != 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 023b4969e338..34b3967ccedb 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 @@ -1684,6 +1684,13 @@ private List getCurrentParseErrors() return limitedParseErrors; } + @Override + public SupervisorTaskAutoScaler createAutoscaler() + { + this.taskAutoScaler = spec.createAutoscaler(this); + return this.taskAutoScaler; + } + @VisibleForTesting public TaskGroup addTaskGroupToActivelyReadingTaskGroup( int taskGroupId, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java index 12519b65ea7c..421bd4558f63 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java @@ -36,7 +36,6 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -62,12 +61,6 @@ public class CostBasedAutoScaler implements SupervisorTaskAutoScaler public static final String IDLE_COST_METRIC = "task/autoScaler/costBased/idleCost"; public static final String OPTIMAL_TASK_COUNT_METRIC = "task/autoScaler/costBased/optimalTaskCount"; - enum CostComputeMode - { - NORMAL, - TASK_ROLLOVER - } - private final String supervisorId; private final SeekableStreamSupervisor supervisor; private final ServiceEmitter emitter; @@ -105,12 +98,8 @@ public CostBasedAutoScaler( @Override public void start() { - Callable scaleAction = () -> computeOptimalTaskCount(this.collectMetrics(), CostComputeMode.NORMAL); - Runnable onSuccessfulScale = () -> { - }; - autoscalerExecutor.scheduleAtFixedRate( - supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, emitter), + supervisor.buildDynamicAllocationTask(this::computeTaskCountForScaleAction, () -> {}, emitter), config.getScaleActionPeriodMillis(), config.getScaleActionPeriodMillis(), TimeUnit.MILLISECONDS @@ -139,7 +128,17 @@ public void reset() @Override public int computeTaskCountForRollover() { - return computeOptimalTaskCount(collectMetrics(), CostComputeMode.TASK_ROLLOVER); + return computeOptimalTaskCount(collectMetrics()); + } + + public int computeTaskCountForScaleAction() + { + CostMetrics costMetrics = collectMetrics(); + final int optimalTaskCount = computeOptimalTaskCount(costMetrics); + final int currentTaskCount = costMetrics.getCurrentTaskCount(); + + // Perform only scale-up actions + return optimalTaskCount >= currentTaskCount ? optimalTaskCount : -1; } public CostBasedAutoScalerConfig getConfig() @@ -160,7 +159,7 @@ public CostBasedAutoScalerConfig getConfig() * * @return optimal task count for scale-up, or -1 if no scaling action needed */ - int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComputeMode) + int computeOptimalTaskCount(CostMetrics metrics) { if (metrics == null) { log.debug("No metrics available yet for supervisorId [%s]", supervisorId); @@ -181,11 +180,7 @@ int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComputeMode } int optimalTaskCount = -1; - CostResult optimalCost = new CostResult( - Double.POSITIVE_INFINITY, - Double.POSITIVE_INFINITY, - Double.POSITIVE_INFINITY - ); + CostResult optimalCost = new CostResult(); for (int taskCount : validTaskCounts) { CostResult costResult = costFunction.computeCost(metrics, taskCount, config); @@ -222,13 +217,7 @@ int computeOptimalTaskCount(CostMetrics metrics, CostComputeMode costComputeMode if (optimalTaskCount == currentTaskCount) { return -1; } - // Perform scale down on task rollover only - if (optimalTaskCount < currentTaskCount) { - if (costComputeMode == CostComputeMode.TASK_ROLLOVER) { - return optimalTaskCount; - } - return -1; - } + // Scale up is performed eagerly. return optimalTaskCount; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java index 1ad946ff5175..42096dd61e17 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostResult.java @@ -25,10 +25,16 @@ */ public class CostResult { + private final double totalCost; private final double lagCost; private final double idleCost; + public CostResult() + { + this(Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY); + } + /** * @param totalCost the weighted sum of lagCost and idleCost * @param lagCost the weighted cost representing expected time (seconds) to recover current lag diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java index 19561f054cb3..977c30a8ff0c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java @@ -70,29 +70,18 @@ public void testComputeValidTaskCounts() // For 100 partitions at 25 tasks (4 partitions/task), valid counts include 25 and 34 int[] validTaskCounts = CostBasedAutoScaler.computeValidTaskCounts(100, 25); - Assert.assertTrue("Should contain current task count", contains(validTaskCounts, 25)); - Assert.assertTrue("Should contain next scale-up option", contains(validTaskCounts, 34)); + Assert.assertTrue("Should contain the current task count", contains(validTaskCounts, 25)); + Assert.assertTrue("Should contain the next scale-up option", contains(validTaskCounts, 34)); - // Edge case: zero partitions returns empty array + // Edge case: zero partitions return an empty array Assert.assertEquals(0, CostBasedAutoScaler.computeValidTaskCounts(0, 10).length); } @Test public void testComputeOptimalTaskCountInvalidInputs() { - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(null, CostBasedAutoScaler.CostComputeMode.NORMAL)); - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(0.0, 10, 0, 0.0), - CostBasedAutoScaler.CostComputeMode.NORMAL - )); - } - - @Test - public void testComputeOptimalTaskCountIdleInIdealRange() - { - // When idle is in ideal range [0.2, 0.6], no scaling should occur - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(5000.0, 25, 100, 0.4), - CostBasedAutoScaler.CostComputeMode.NORMAL - )); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(null)); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(0.0, 10, 0, 0.0))); } @Test @@ -100,31 +89,20 @@ public void testComputeOptimalTaskCountScaling() { // High idle (underutilized) - should scale down // With high idle (0.8), the algorithm evaluates lower task counts and finds they have lower idle cost - int scaleDownResult = autoScaler.computeOptimalTaskCount(createMetrics(100.0, 25, 100, 0.8), - CostBasedAutoScaler.CostComputeMode.NORMAL - ); + int scaleDownResult = autoScaler.computeOptimalTaskCount(createMetrics(100.0, 25, 100, 0.8)); Assert.assertTrue("Should scale down when idle > 0.6", scaleDownResult < 25); } @Test public void testComputeOptimalTaskCountLowIdleDoesNotScaleUpWithBalancedWeights() { - // With corrected idle ratio model and marginal lag model, low idle does not - // automatically trigger scale-up. The algorithm is conservative because: - // 1. Scale-up increases idle cost (more tasks = more idle per task with fixed load) - // 2. Marginal lag model means only ADDITIONAL tasks work on backlog - // - // This is intentional: the idle-heavy weights (0.4 idle) make the algorithm - // favor stability over aggressive scaling - int result = autoScaler.computeOptimalTaskCount(createMetrics(1000.0, 25, 100, 0.1), - CostBasedAutoScaler.CostComputeMode.NORMAL - ); + // With a corrected idle ratio model and marginal lag model, low idle does not automatically trigger scale-up. + int result = autoScaler.computeOptimalTaskCount(createMetrics(1000.0, 25, 100, 0.1)); - // Algorithm evaluates costs and may find current count optimal - // or may scale down if idle cost reduction outweighs lag increase + // Algorithm evaluates costs and may find the current count optimal + // or may scale down if idle cost reduction outweighs lag increase. Assert.assertTrue( - "With low idle and balanced weights, algorithm should not scale up aggressively", - result == -1 || result <= 25 + "With low idle and balanced weights, algorithm should not scale up aggressively", result <= 25 ); } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java index 30a0d7a723e4..44dcdc9d7147 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java @@ -24,6 +24,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler; import org.apache.druid.segment.incremental.ParseExceptionReport; import javax.annotation.Nullable; @@ -86,6 +87,11 @@ default Boolean isHealthy() return null; // default implementation for interface compatability; returning null since true or false is misleading } + default SupervisorTaskAutoScaler createAutoscaler() + { + return null; + } + /** * Resets any stored metadata by the supervisor. * @param dataSourceMetadata optional dataSource metadata. From 7788e3896f599267404eb8126c4ade7c8b7bf146 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 15:04:34 +0200 Subject: [PATCH 09/20] Formatting --- .../supervisor/SeekableStreamSupervisor.java | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 34b3967ccedb..34adf69a44f5 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 @@ -3358,39 +3358,39 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() ) ) - .forEach(entry -> { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); - final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); - if (stopTasksEarly || group.getHandoffEarly()) { - // If handoffEarly has been set, stop tasks irrespective of stopTaskCount + .forEach(entry -> { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); + final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); + if (stopTasksEarly || group.getHandoffEarly()) { + // If handoffEarly has been set, stop tasks irrespective of stopTaskCount + log.info( + "Stopping taskGroup[%d] early after running for duration[%s].", + groupId, runDuration + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + if (group.getHandoffEarly()) { + numStoppedTasks.getAndIncrement(); + } + } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { + // Stop this task group if it has run longer than the configured duration + // and the pending task groups are less than the configured stop task count. + int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() + .mapToInt(List::size).sum(); + if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { log.info( - "Stopping taskGroup[%d] early after running for duration[%s].", - groupId, runDuration + "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", + groupId, runDuration, ioConfig.getTaskDuration() ); futureGroupIds.add(groupId); futures.add(checkpointTaskGroup(group, true)); - if (group.getHandoffEarly()) { - numStoppedTasks.getAndIncrement(); - } - } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { - // Stop this task group if it has run longer than the configured duration - // and the pending task groups are less than the configured stop task count. - int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() - .mapToInt(List::size).sum(); - if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { - log.info( - "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", - groupId, runDuration, ioConfig.getTaskDuration() - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - numStoppedTasks.getAndIncrement(); - } + numStoppedTasks.getAndIncrement(); } - }); + } + }); List>> results = coalesceAndAwait(futures); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); From 5936fd0f0b6b92562417dd309ac65d2c678d3ab9 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 15:06:10 +0200 Subject: [PATCH 10/20] Formattin --- .../supervisor/SeekableStreamSupervisor.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 34adf69a44f5..1dcad385d69b 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 @@ -3354,17 +3354,17 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing activelyReadingTaskGroups.entrySet().stream().sorted( - Comparator.comparingLong( - taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() - ) - ) - .forEach(entry -> { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); - final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); - if (stopTasksEarly || group.getHandoffEarly()) { + Comparator.comparingLong( + taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() + ) + ) + .forEach(entry -> { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); + final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); + if (stopTasksEarly || group.getHandoffEarly()) { // If handoffEarly has been set, stop tasks irrespective of stopTaskCount log.info( "Stopping taskGroup[%d] early after running for duration[%s].", From 640287c56c9e45086462037627c11e6545651617 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 15:09:31 +0200 Subject: [PATCH 11/20] Revert "Formattin" This reverts commit 5936fd0f0b6b92562417dd309ac65d2c678d3ab9. --- .../supervisor/SeekableStreamSupervisor.java | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 1dcad385d69b..34adf69a44f5 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 @@ -3354,17 +3354,17 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing activelyReadingTaskGroups.entrySet().stream().sorted( - Comparator.comparingLong( - taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() - ) - ) - .forEach(entry -> { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); - final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); - if (stopTasksEarly || group.getHandoffEarly()) { + Comparator.comparingLong( + taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() + ) + ) + .forEach(entry -> { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); + final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); + if (stopTasksEarly || group.getHandoffEarly()) { // If handoffEarly has been set, stop tasks irrespective of stopTaskCount log.info( "Stopping taskGroup[%d] early after running for duration[%s].", From 2722e0db6ad5c61d9f621fd137a800f5a6e7c056 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 15:09:31 +0200 Subject: [PATCH 12/20] Revert "Formatting" This reverts commit 7788e3896f599267404eb8126c4ade7c8b7bf146. --- .../supervisor/SeekableStreamSupervisor.java | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 34adf69a44f5..34b3967ccedb 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 @@ -3358,39 +3358,39 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() ) ) - .forEach(entry -> { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); - final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); - if (stopTasksEarly || group.getHandoffEarly()) { - // If handoffEarly has been set, stop tasks irrespective of stopTaskCount - log.info( - "Stopping taskGroup[%d] early after running for duration[%s].", - groupId, runDuration - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - if (group.getHandoffEarly()) { - numStoppedTasks.getAndIncrement(); - } - } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { - // Stop this task group if it has run longer than the configured duration - // and the pending task groups are less than the configured stop task count. - int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() - .mapToInt(List::size).sum(); - if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { + .forEach(entry -> { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); + final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); + if (stopTasksEarly || group.getHandoffEarly()) { + // If handoffEarly has been set, stop tasks irrespective of stopTaskCount log.info( - "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", - groupId, runDuration, ioConfig.getTaskDuration() + "Stopping taskGroup[%d] early after running for duration[%s].", + groupId, runDuration ); futureGroupIds.add(groupId); futures.add(checkpointTaskGroup(group, true)); - numStoppedTasks.getAndIncrement(); + if (group.getHandoffEarly()) { + numStoppedTasks.getAndIncrement(); + } + } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { + // Stop this task group if it has run longer than the configured duration + // and the pending task groups are less than the configured stop task count. + int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() + .mapToInt(List::size).sum(); + if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { + log.info( + "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", + groupId, runDuration, ioConfig.getTaskDuration() + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + numStoppedTasks.getAndIncrement(); + } } - } - }); + }); List>> results = coalesceAndAwait(futures); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); From 2830ea1a568c55ae44570cfd674d355650e7c1f7 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 15:12:42 +0200 Subject: [PATCH 13/20] Format Signed-off-by: Sasha Syrotenko --- ...eStreamSupervisor_Stream_Ingestion_Flow.md | 910 ++++++++++++++++++ 1 file changed, 910 insertions(+) create mode 100644 docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md diff --git a/docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md b/docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md new file mode 100644 index 000000000000..4df40d77ada3 --- /dev/null +++ b/docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md @@ -0,0 +1,910 @@ +# SeekableStreamSupervisor Stream Ingestion Flow + +This document provides a comprehensive code flow diagram of the stream ingestion process in Apache Druid's `SeekableStreamSupervisor`, with detailed analysis of Kafka/Kinesis partition assignments. + +## Table of Contents + +1. [Architecture Overview](#architecture-overview) +2. [Class Hierarchy](#class-hierarchy) +3. [Supervisor Lifecycle](#supervisor-lifecycle) +4. [Main Run Loop Flow](#main-run-loop-flow) +5. [Partition Discovery and Assignment](#partition-discovery-and-assignment) +6. [Task Group Management](#task-group-management) +7. [Task Creation Flow](#task-creation-flow) +8. [Task Execution Flow](#task-execution-flow) +9. [Checkpointing and Publishing](#checkpointing-and-publishing) +10. [Kafka vs Kinesis Differences](#kafka-vs-kinesis-differences) + +--- + +## Architecture Overview + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ OVERLORD (Leader) │ +│ ┌─────────────────────────────────────────────────────────────────────┐ │ +│ │ SupervisorManager │ │ +│ │ ┌─────────────────────────────────────────────────────────────┐ │ │ +│ │ │ SeekableStreamSupervisor │ │ │ +│ │ │ ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ │ │ │ +│ │ │ │ KafkaSuperv. │ │ KinesisSuperv │ │ RecordSupplier│ │ │ │ +│ │ │ └───────────────┘ └───────────────┘ └───────────────┘ │ │ │ +│ │ │ │ │ │ +│ │ │ ┌─────────────────────────────────────────────────────┐ │ │ │ +│ │ │ │ NoticesQueue │ │ │ │ +│ │ │ │ [RunNotice, CheckpointNotice, ShutdownNotice, ...] │ │ │ │ +│ │ │ └─────────────────────────────────────────────────────┘ │ │ │ +│ │ └─────────────────────────────────────────────────────────────┘ │ │ +│ └─────────────────────────────────────────────────────────────────────┘ │ +│ │ │ +│ ▼ │ +│ ┌─────────────────────────────────────────────────────────────────────┐ │ +│ │ TaskQueue │ │ +│ │ ┌──────────────────────────────────────────────────────────┐ │ │ +│ │ │ SeekableStreamIndexTask (Kafka/Kinesis IndexTask) │ │ │ +│ │ └──────────────────────────────────────────────────────────┘ │ │ +│ └─────────────────────────────────────────────────────────────────────┘ │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────────┐ +│ MIDDLE MANAGER / PEON │ +│ ┌─────────────────────────────────────────────────────────────────────┐ │ +│ │ SeekableStreamIndexTaskRunner │ │ +│ │ ┌───────────────────────────────────────────────────────────────┐ │ │ +│ │ │ Main Ingestion Loop: │ │ │ +│ │ │ - poll() records from stream │ │ │ +│ │ │ - parse records → InputRows │ │ │ +│ │ │ - add to Appenderator │ │ │ +│ │ │ - checkpoint/publish segments │ │ │ +│ │ └───────────────────────────────────────────────────────────────┘ │ │ +│ └─────────────────────────────────────────────────────────────────────┘ │ +└─────────────────────────────────────────────────────────────────────────────┘ +``` + +--- + +## Class Hierarchy + +``` +StreamSupervisor (interface) + └── SeekableStreamSupervisor + ├── KafkaSupervisor + └── KinesisSupervisor + +SeekableStreamIndexTask + ├── KafkaIndexTask + └── KinesisIndexTask + +RecordSupplier + ├── KafkaRecordSupplier + └── KinesisRecordSupplier +``` + +### Key Classes and Files + +| Class | Location | Purpose | +|---------------------------------|-----------------------------------------------------------------------|--------------------------------------------| +| `SeekableStreamSupervisor` | `indexing-service/.../supervisor/SeekableStreamSupervisor.java` | Abstract base class for stream supervisors | +| `KafkaSupervisor` | `extensions-core/kafka-indexing-service/.../KafkaSupervisor.java` | Kafka-specific supervisor implementation | +| `KinesisSupervisor` | `extensions-core/kinesis-indexing-service/.../KinesisSupervisor.java` | Kinesis-specific supervisor implementation | +| `SeekableStreamIndexTask` | `indexing-service/.../SeekableStreamIndexTask.java` | Abstract base for indexing tasks | +| `SeekableStreamIndexTaskRunner` | `indexing-service/.../SeekableStreamIndexTaskRunner.java` | Task runner with main ingestion loop | +| `RecordSupplier` | `indexing-service/.../common/RecordSupplier.java` | Interface for stream consumer | + +--- + +## Supervisor Lifecycle + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ SUPERVISOR LIFECYCLE │ +└─────────────────────────────────────────────────────────────────────────────┘ + + ┌──────────────┐ + │ start() │ + └──────┬───────┘ + │ + ▼ + ┌──────────────┐ retry (up to 20 times) + │ tryInit() │ ─────────────────────────┐ + └──────┬───────┘ │ + │ success │ + ▼ │ + ┌──────────────────────────────────────┐ │ + │ setupRecordSupplier() │ │ + │ (Creates Kafka/Kinesis consumer) │◄─┘ + └──────┬───────────────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ Start Notice Processing Thread │ + │ (exec.submit -> notices.poll()) │ + └──────┬───────────────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ Schedule Periodic RunNotice │ + │ (scheduledExec.scheduleAtFixedRate) │ + │ Period: ioConfig.period (default 1m)│ + └──────┬───────────────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ Schedule Reporting │ + │ (Lag metrics, offset updates) │ + └──────┬───────────────────────────────┘ + │ + ▼ + ┌──────────────┐ + │ RUNNING │ + └──────────────┘ +``` + +### Key Initialization Code Path +**File:** `SeekableStreamSupervisor.java:1276-1357` + +```java +public void tryInit() { + recordSupplier = setupRecordSupplier(); // Creates Kafka/Kinesis consumer + + exec.submit(() -> { + while (!stopped) { + Notice notice = notices.poll(pollTimeout); + notice.handle(); // Process RunNotice, CheckpointNotice, etc. + } + }); + + scheduledExec.scheduleAtFixedRate( + buildRunTask(), // Adds RunNotice to queue + ioConfig.getStartDelay().getMillis(), + ioConfig.getPeriod().getMillis(), + TimeUnit.MILLISECONDS + ); +} +``` + +--- + +## Main Run Loop Flow + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ runInternal() FLOW │ +│ SeekableStreamSupervisor.java:1733 │ +└─────────────────────────────────────────────────────────────────────────────┘ + + ┌──────────────────────────────────────┐ + │ RunNotice.handle() │ + │ │ │ + │ ▼ │ + │ runInternal() │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 1. possiblyRegisterListener() │ Register TaskRunnerListener + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 2. updatePartitionDataFromStream() │ ◄── PARTITION DISCOVERY + │ - recordSupplier.getPartitionIds()│ + │ - Assign partitions to groups │ + │ - Handle new/expired partitions │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 3. discoverTasks() │ ◄── TASK DISCOVERY + │ - Find existing tasks │ + │ - Create TaskGroup for each │ + │ - Verify checkpoints │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 4. updateTaskStatus() │ + │ - Get startTime from tasks │ + │ - Update task status │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 5. checkTaskDuration() │ ◄── TASK LIFECYCLE + │ - If task exceeded duration │ + │ - Signal tasks to finish │ + │ - Move to pendingCompletion │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 6. checkPendingCompletionTasks() │ + │ - Monitor publishing tasks │ + │ - Handle timeouts/failures │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 7. checkCurrentTaskState() │ + │ - Verify tasks are healthy │ + │ - Kill failed tasks │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 8. checkIfStreamInactiveAndTurn... │ + │ - Idle detection │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 9. createNewTasks() │ ◄── TASK CREATION + │ - Create TaskGroups for groups │ + │ without active tasks │ + │ - Create replica tasks │ + └──────────────────────────────────────┘ +``` + +--- + +## Partition Discovery and Assignment + +### Partition Discovery Flow + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ updatePartitionDataFromStream() │ +│ SeekableStreamSupervisor.java:2849 │ +└─────────────────────────────────────────────────────────────────────────────┘ + + ┌──────────────────────────────────────┐ + │ recordSupplier.getPartitionIds() │ + │ (Kafka: KafkaConsumer.partitionsFor)│ + │ (Kinesis: listShards API) │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ Get stored metadata offsets │ + │ indexerMetadataStorageCoordinator │ + │ .retrieveDataSourceMetadata() │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ Filter partitions: │ + │ - Remove closed partitions │ + │ - Remove expired partitions │ + │ - activePartitionsIdsFromSupplier │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ For each active partition: │ + │ ┌────────────────────────────────┐ │ + │ │ taskGroupId = │ │ + │ │ getTaskGroupIdForPartition() │ │ + │ │ │ │ + │ │ Kafka: partition % taskCount │ │ + │ │ Kinesis: index % taskCount │ │ + │ └────────────────────────────────┘ │ + │ │ + │ partitionGroups.put(taskGroupId, │ + │ partition) │ + │ partitionOffsets.put(partition, │ + │ NOT_SET) │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ If partitions changed: │ + │ - assignRecordSupplierToPartitionIds│ + │ - Set earlyStopTime for │ + │ repartition transition │ + └──────────────────────────────────────┘ +``` + +### Partition Assignment Algorithm + +#### Kafka Partition Assignment +**File:** `KafkaSupervisor.java:141-149` + +```java +@Override +protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) { + Integer taskCount = spec.getIoConfig().getTaskCount(); + if (partitionId.isMultiTopicPartition()) { + // Multi-topic: hash based assignment + return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; + } else { + // Single topic: simple modulo + return partitionId.partition() % taskCount; + } +} +``` + +``` +Example: 12 partitions, taskCount = 3 + +Partition 0 → TaskGroup 0 (0 % 3 = 0) +Partition 1 → TaskGroup 1 (1 % 3 = 1) +Partition 2 → TaskGroup 2 (2 % 3 = 2) +Partition 3 → TaskGroup 0 (3 % 3 = 0) +Partition 4 → TaskGroup 1 (4 % 3 = 1) +... +Partition 11 → TaskGroup 2 (11 % 3 = 2) + +TaskGroup 0: [P0, P3, P6, P9] +TaskGroup 1: [P1, P4, P7, P10] +TaskGroup 2: [P2, P5, P8, P11] +``` + +#### Kinesis Shard Assignment +**File:** `KinesisSupervisor.java:213-226` + +```java +@Override +protected int getTaskGroupIdForPartition(String partitionId) { + return getTaskGroupIdForPartitionWithProvidedList(partitionId, partitionIds); +} + +private int getTaskGroupIdForPartitionWithProvidedList(String partitionId, List availablePartitions) { + int index = availablePartitions.indexOf(partitionId); + if (index < 0) { + return index; + } + return availablePartitions.indexOf(partitionId) % spec.getIoConfig().getTaskCount(); +} +``` + +``` +Example: 6 shards, taskCount = 2 + +Shard List (ordered): [shard-000, shard-001, shard-002, shard-003, shard-004, shard-005] + +shard-000 (index 0) → TaskGroup 0 (0 % 2 = 0) +shard-001 (index 1) → TaskGroup 1 (1 % 2 = 1) +shard-002 (index 2) → TaskGroup 0 (2 % 2 = 0) +shard-003 (index 3) → TaskGroup 1 (3 % 2 = 1) +shard-004 (index 4) → TaskGroup 0 (4 % 2 = 0) +shard-005 (index 5) → TaskGroup 1 (5 % 2 = 1) + +TaskGroup 0: [shard-000, shard-002, shard-004] +TaskGroup 1: [shard-001, shard-003, shard-005] +``` + +--- + +## Task Group Management + +### Data Structures + +**File:** `SeekableStreamSupervisor.java:845-865` + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ SUPERVISOR DATA STRUCTURES │ +└─────────────────────────────────────────────────────────────────────────────┘ + +activelyReadingTaskGroups: ConcurrentHashMap +├── TaskGroup 0 +│ ├── groupId: 0 +│ ├── startingSequences: {P0→offset1, P3→offset2, P6→offset3} +│ ├── tasks: ConcurrentHashMap +│ │ ├── "task-abc-123" → TaskData{status, startTime, currentSequences} +│ │ └── "task-abc-456" → TaskData{status, startTime, currentSequences} +│ ├── checkpointSequences: TreeMap> +│ ├── minimumMessageTime, maximumMessageTime +│ ├── exclusiveStartSequenceNumberPartitions +│ └── baseSequenceName: "seq_kafka_topic_abc123..." +├── TaskGroup 1 +│ └── ... +└── TaskGroup 2 + └── ... + +pendingCompletionTaskGroups: ConcurrentHashMap> +├── TaskGroup 0 → [TaskGroup(publishing), TaskGroup(publishing)] +└── ... + +partitionGroups: ConcurrentHashMap> +├── 0 → {P0, P3, P6, P9} +├── 1 → {P1, P4, P7, P10} +└── 2 → {P2, P5, P8, P11} + +partitionOffsets: ConcurrentHashMap +├── P0 → 12345 +├── P1 → 67890 +└── ... + +partitionIds: CopyOnWriteArrayList +└── [P0, P1, P2, P3, P4, P5, P6, P7, P8, P9, P10, P11] +``` + +### TaskGroup Lifecycle + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ TaskGroup STATE MACHINE │ +└─────────────────────────────────────────────────────────────────────────────┘ + + ┌─────────────────────┐ + │ NO TaskGroup │ + │ (partitionGroup │ + │ exists, but no │ + │ TaskGroup yet) │ + └──────────┬──────────┘ + │ createNewTasks() + ▼ + ┌─────────────────────┐ + │ activelyReading │ ◄─────────────────────┐ + │ TaskGroups │ │ + │ │ New task created │ + │ - Tasks are reading │ after previous │ + │ - Ingesting data │ completes │ + └──────────┬──────────┘ │ + │ │ + │ checkTaskDuration(): │ + │ task.startTime + taskDuration │ + │ has passed │ + ▼ │ + ┌─────────────────────┐ │ + │ checkpointTaskGroup │ │ + │ │ │ + │ 1. Pause all tasks │ │ + │ 2. Get highest │ │ + │ offsets │ │ + │ 3. Set end offsets │ │ + │ 4. Resume tasks │ │ + └──────────┬──────────┘ │ + │ │ + ▼ │ + ┌─────────────────────┐ │ + │ pendingCompletion │ │ + │ TaskGroups │ │ + │ │ │ + │ - Tasks are │ │ + │ publishing │ │ + │ - Waiting for │ │ + │ handoff │ │ + └──────────┬──────────┘ │ + │ │ + │ Task SUCCESS │ + ▼ │ + ┌─────────────────────┐ │ + │ TaskGroup removed │───────────────────────┘ + │ from pending │ + │ completion │ + │ │ + │ Offsets updated in │ + │ partitionOffsets │ + └─────────────────────┘ +``` + +--- + +## Task Creation Flow + +**File:** `SeekableStreamSupervisor.java:3854-3975` + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ createNewTasks() FLOW │ +└─────────────────────────────────────────────────────────────────────────────┘ + + ┌──────────────────────────────────────┐ + │ For each groupId in partitionGroups │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ if (!activelyReadingTaskGroups │ Check if TaskGroup + │ .containsKey(groupId)) │ already exists + └──────────────┬───────────────────────┘ + │ No TaskGroup exists + ▼ + ┌──────────────────────────────────────┐ + │ Generate starting sequences: │ + │ generateStartingSequencesFor...() │ + │ │ + │ 1. Check partitionOffsets │ + │ 2. If NOT_SET, check metadata store │ + │ 3. If not in metadata, get from │ + │ stream (earliest/latest) │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ Create new TaskGroup: │ + │ new TaskGroup( │ + │ groupId, │ + │ startingSequences, │ + │ minimumMessageTime, │ + │ maximumMessageTime, │ + │ exclusiveStartSequencePartitions │ + │ ) │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ activelyReadingTaskGroups.put(...) │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ For each TaskGroup: │ + │ if (replicas > tasks.size()) │ + │ createTasksForGroup(groupId, │ + │ replicas - tasks.size()) │ + └──────────────┬───────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────────────────────────────────┐ + │ createTasksForGroup(): │ + │ │ + │ 1. Create SeekableStreamIndexTaskIOConfig │ + │ - startPartitions: group.startingSequences │ + │ - endPartitions: END_OF_PARTITION marker for each partition │ + │ │ + │ 2. createIndexTasks() [Kafka/Kinesis specific] │ + │ - Creates KafkaIndexTask or KinesisIndexTask │ + │ - taskId: random ID with baseSequenceName prefix │ + │ │ + │ 3. taskQueue.add(indexTask) │ + │ - Task is submitted to Overlord's task queue │ + └──────────────────────────────────────────────────────────────────┘ +``` + +--- + +## Task Execution Flow + +### SeekableStreamIndexTaskRunner Main Loop + +**File:** `SeekableStreamIndexTaskRunner.java:399-700` + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ SeekableStreamIndexTaskRunner.run() │ +└─────────────────────────────────────────────────────────────────────────────┘ + + ┌──────────────────────────────────────┐ + │ 1. Initialize │ + │ - Create StreamChunkParser │ + │ - initializeSequences() │ + │ - Register with ChatHandler │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 2. Setup RecordSupplier │ + │ task.newTaskRecordSupplier() │ + │ - KafkaRecordSupplier │ + │ - KinesisRecordSupplier │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 3. Setup Appenderator + Driver │ + │ - newAppenderator() │ + │ - newDriver() │ + │ - driver.startJob() │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 4. Restore/Initialize Offsets │ + │ - currOffsets from metadata │ + │ - or from startSequenceNumbers │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 5. Assign & Seek Partitions │ + │ assignPartitions(recordSupplier) │ + │ seekToStartingSequence() │ + └──────────────┬───────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────────┐ +│ MAIN INGESTION LOOP │ +│ │ +│ while (stillReading) { │ +│ │ │ +│ ├─► possiblyPause() // Handle pause/resume from supervisor │ +│ │ │ +│ ├─► if (stopRequested) // Check if supervisor signaled stop │ +│ │ status = PUBLISHING │ +│ │ break │ +│ │ │ +│ ├─► maybePersistAndPublish... // Checkpoint if needed │ +│ │ │ +│ ├─► getRecords(recordSupplier) │ +│ │ └── recordSupplier.poll(timeout) │ +│ │ ├── Kafka: KafkaConsumer.poll() │ +│ │ └── Kinesis: getRecords() API │ +│ │ │ +│ └─► for (record : records) { │ +│ │ │ +│ ├─► verifyRecordInRange() // Check record is in range │ +│ │ │ +│ ├─► parser.parse(record) // Parse → InputRows │ +│ │ │ +│ ├─► driver.add(row, ...) // Add to Appenderator │ +│ │ │ │ +│ │ └─► if (isPushRequired) // Segment is full │ +│ │ driver.persist() │ +│ │ │ +│ └─► Update currOffsets[partition] = record.sequenceNumber │ +│ } │ +│ } │ +│ │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ + ▼ + ┌──────────────────────────────────────┐ + │ 6. Publishing Phase │ + │ status = PUBLISHING │ + │ │ + │ - Persist remaining data │ + │ - Publish segments │ + │ - Update metadata │ + │ - Handoff segments │ + └──────────────────────────────────────┘ +``` + +--- + +## Checkpointing and Publishing + +### Checkpoint Flow + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ CHECKPOINTING MECHANISM │ +└─────────────────────────────────────────────────────────────────────────────┘ + +Supervisor Side (SeekableStreamSupervisor): +━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━ + + ┌──────────────────────────────────────┐ + │ checkTaskDuration() │ + │ (Task has run for taskDuration) │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ checkpointTaskGroup(group, true) │ (finalize=true) + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 1. Pause all tasks: │ + │ taskClient.pauseAsync(taskId) │ + │ → Returns current offsets │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 2. Compute highest offsets │ + │ across all replica tasks │ + │ endOffsets = max(all offsets) │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 3. Set end offsets and finalize: │ + │ taskClient.setEndOffsetsAsync( │ + │ taskId, endOffsets, true) │ + │ → Task will finish and publish │ + └──────────────┬───────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ 4. Move TaskGroup to │ + │ pendingCompletionTaskGroups │ + │ │ + │ 5. Update partitionOffsets with │ + │ endOffsets for next TaskGroup │ + └──────────────────────────────────────┘ + + +Task Side (SeekableStreamIndexTaskRunner): +━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━ + + ┌──────────────────────────────────────┐ + │ POST /pause │ + │ → possiblyPause() │ + │ → status = PAUSED │ + │ → Return currOffsets │ + └──────────────────────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ POST /offsets/end?finish=true │ + │ → setEndOffsets(offsets, true) │ + │ → endOffsets = offsets │ + │ → Resume reading │ + └──────────────────────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ Continue reading until │ + │ currOffset >= endOffset │ + │ for all partitions │ + └──────────────────────────────────────┘ + │ + ┌──────────────▼───────────────────────┐ + │ Transition to PUBLISHING: │ + │ - Persist remaining data │ + │ - Publish segments │ + │ - Checkpoint metadata │ + │ - Handoff to Historical │ + └──────────────────────────────────────┘ +``` + +### Metadata Storage + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ METADATA STORAGE │ +│ (druid_dataSource table) │ +└─────────────────────────────────────────────────────────────────────────────┘ + +dataSource: "my_datasource" +supervisorId: "supervisor_my_datasource" + +commit_metadata_payload: +{ + "type": "kafka", // or "kinesis" + "partitions": { + "type": "end", + "stream": "my_topic", + "partitionSequenceNumberMap": { + "0": 12345, + "1": 67890, + "2": 11111, + ... + } + } +} + +This metadata is used to: +1. Resume from the last committed offset on restart +2. Ensure exactly-once semantics with segment metadata +3. Coordinate between supervisor and tasks +``` + +--- + +## Kafka vs Kinesis Differences + +| Aspect | Kafka | Kinesis | +|--------|-------|---------| +| **Partition ID Type** | `KafkaTopicPartition` (topic + partition int) | `String` (shard ID) | +| **Sequence Type** | `Long` (offset) | `String` (sequence number) | +| **Partition Assignment** | `partition % taskCount` | `indexOf(shard) % taskCount` | +| **Exclusive Start** | Not used (`useExclusiveStartingSequence = false`) | Used (`useExclusiveStartingSequence = true`) | +| **Shard/Partition Expiration** | No (partitions are permanent) | Yes (shards can expire) | +| **Lag Metric** | Record count lag | Time lag (milliseconds) | +| **End of Shard** | N/A | `KinesisSequenceNumber.END_OF_SHARD_MARKER` | +| **RecordSupplier** | `KafkaRecordSupplier` (wraps KafkaConsumer) | `KinesisRecordSupplier` (uses AWS SDK) | + +### Kafka-Specific Features + +**File:** `KafkaSupervisor.java` + +```java +// Multi-topic support with regex patterns +private final Pattern pattern; // For topic regex matching + +// Partition assignment for multi-topic +if (partitionId.isMultiTopicPartition()) { + return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; +} + +// Record lag (offset-based) +@Override +protected Map getPartitionRecordLag() { + return latestSequenceFromStream - currentOffsets; +} +``` + +### Kinesis-Specific Features + +**File:** `KinesisSupervisor.java` + +```java +// Shard expiration handling +@Override +protected boolean supportsPartitionExpiration() { + return true; +} + +@Override +protected boolean isEndOfShard(String seqNum) { + return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum); +} + +// Shard rebalancing on expiration +@Override +protected Map> recomputePartitionGroupsForExpiration(Set availablePartitions) { + // Redistribute shards evenly across task groups + List availablePartitionsList = new ArrayList<>(availablePartitions); + for (String partition : availablePartitions) { + int newTaskGroupId = indexOf(partition) % taskCount; + newPartitionGroups.get(newTaskGroupId).add(partition); + } + return newPartitionGroups; +} + +// Time-based lag (no record count available) +@Override +protected Map getPartitionTimeLag() { + return currentPartitionTimeLag; // milliseconds behind latest +} +``` + +--- + +## Key Configuration Parameters + +### Supervisor IOConfig + +| Parameter | Default | Description | +|-----------|---------|-------------| +| `taskCount` | 1 | Number of task groups (parallel readers) | +| `replicas` | 1 | Number of replica tasks per group | +| `taskDuration` | PT1H | How long each task runs before publishing | +| `completionTimeout` | PT30M | Timeout for task completion | +| `startDelay` | PT5S | Delay before first run | +| `period` | PT30S | How often supervisor runs | +| `useEarliestSequenceNumber` | false | Start from earliest or latest | +| `maxAllowedStops` | 1 | Maximum concurrent task group stops | + +### Supervisor TuningConfig + +| Parameter | Default | Description | +|-----------|---------|-------------| +| `workerThreads` | null | Thread pool size for task communication | +| `chatThreads` | null | Threads for HTTP endpoints | +| `chatRetries` | 8 | Retries for task communication | +| `httpTimeout` | PT10S | HTTP request timeout | +| `shutdownTimeout` | PT80S | Shutdown wait time | +| `repartitionTransitionDuration` | PT2M | Cooldown after partition changes | + +--- + +## Summary Diagram + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ COMPLETE STREAM INGESTION FLOW │ +└─────────────────────────────────────────────────────────────────────────────┘ + +Stream (Kafka/Kinesis) Overlord MiddleManager/Peon +━━━━━━━━━━━━━━━━━━━━━━━━━ ━━━━━━━━━━━━━━━━━━━━━━━━ ━━━━━━━━━━━━━━━━━━━━━━━ + +┌───────────────────┐ +│ Topic/Stream │ +│ ┌───┬───┬───┬───┐ │ ┌──────────────────────┐ +│ │P0 │P1 │P2 │P3 │ │─────►│ SeekableStream │ +│ └───┴───┴───┴───┘ │ │ Supervisor │ +└───────────────────┘ │ │ + │ 1. getPartitionIds() │ + │ 2. Assign to groups │ + │ 3. Create TaskGroups │ + └──────────┬───────────┘ + │ + ▼ + ┌──────────────────────┐ + │ createIndexTasks() │ + │ │ + │ TaskGroup 0: │ + │ Task-0a (replica) │────────┐ + │ Task-0b (replica) │────────┤ + │ │ │ + │ TaskGroup 1: │ │ + │ Task-1a (replica) │────────┤ + └──────────────────────┘ │ + ▼ + ┌─────────────────────┐ + │ SeekableStream │ + │ IndexTaskRunner │ + │ │ + │ while(reading) { │ +┌───────────────────┐ │ poll(stream) │ +│ RecordSupplier │◄─────────────────────────│ parse(records) │ +│ (poll records) │ │ add(appenderator) │ +└───────────────────┘ │ } │ + │ │ + │ publish segments │──────► Deep Storage + │ update metadata │──────► Metadata Store + │ handoff │──────► Historical + └─────────────────────┘ +``` + +--- + +--- + From fd654e2338e56a807c099e489338ed51c2727a08 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 15:13:28 +0200 Subject: [PATCH 14/20] Remove obsolete file --- ...eStreamSupervisor_Stream_Ingestion_Flow.md | 910 ------------------ .../supervisor/SeekableStreamSupervisor.java | 74 +- 2 files changed, 37 insertions(+), 947 deletions(-) delete mode 100644 docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md diff --git a/docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md b/docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md deleted file mode 100644 index 4df40d77ada3..000000000000 --- a/docs/SeekableStreamSupervisor_Stream_Ingestion_Flow.md +++ /dev/null @@ -1,910 +0,0 @@ -# SeekableStreamSupervisor Stream Ingestion Flow - -This document provides a comprehensive code flow diagram of the stream ingestion process in Apache Druid's `SeekableStreamSupervisor`, with detailed analysis of Kafka/Kinesis partition assignments. - -## Table of Contents - -1. [Architecture Overview](#architecture-overview) -2. [Class Hierarchy](#class-hierarchy) -3. [Supervisor Lifecycle](#supervisor-lifecycle) -4. [Main Run Loop Flow](#main-run-loop-flow) -5. [Partition Discovery and Assignment](#partition-discovery-and-assignment) -6. [Task Group Management](#task-group-management) -7. [Task Creation Flow](#task-creation-flow) -8. [Task Execution Flow](#task-execution-flow) -9. [Checkpointing and Publishing](#checkpointing-and-publishing) -10. [Kafka vs Kinesis Differences](#kafka-vs-kinesis-differences) - ---- - -## Architecture Overview - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ OVERLORD (Leader) │ -│ ┌─────────────────────────────────────────────────────────────────────┐ │ -│ │ SupervisorManager │ │ -│ │ ┌─────────────────────────────────────────────────────────────┐ │ │ -│ │ │ SeekableStreamSupervisor │ │ │ -│ │ │ ┌───────────────┐ ┌───────────────┐ ┌───────────────┐ │ │ │ -│ │ │ │ KafkaSuperv. │ │ KinesisSuperv │ │ RecordSupplier│ │ │ │ -│ │ │ └───────────────┘ └───────────────┘ └───────────────┘ │ │ │ -│ │ │ │ │ │ -│ │ │ ┌─────────────────────────────────────────────────────┐ │ │ │ -│ │ │ │ NoticesQueue │ │ │ │ -│ │ │ │ [RunNotice, CheckpointNotice, ShutdownNotice, ...] │ │ │ │ -│ │ │ └─────────────────────────────────────────────────────┘ │ │ │ -│ │ └─────────────────────────────────────────────────────────────┘ │ │ -│ └─────────────────────────────────────────────────────────────────────┘ │ -│ │ │ -│ ▼ │ -│ ┌─────────────────────────────────────────────────────────────────────┐ │ -│ │ TaskQueue │ │ -│ │ ┌──────────────────────────────────────────────────────────┐ │ │ -│ │ │ SeekableStreamIndexTask (Kafka/Kinesis IndexTask) │ │ │ -│ │ └──────────────────────────────────────────────────────────┘ │ │ -│ └─────────────────────────────────────────────────────────────────────┘ │ -└─────────────────────────────────────────────────────────────────────────────┘ - │ - ▼ -┌─────────────────────────────────────────────────────────────────────────────┐ -│ MIDDLE MANAGER / PEON │ -│ ┌─────────────────────────────────────────────────────────────────────┐ │ -│ │ SeekableStreamIndexTaskRunner │ │ -│ │ ┌───────────────────────────────────────────────────────────────┐ │ │ -│ │ │ Main Ingestion Loop: │ │ │ -│ │ │ - poll() records from stream │ │ │ -│ │ │ - parse records → InputRows │ │ │ -│ │ │ - add to Appenderator │ │ │ -│ │ │ - checkpoint/publish segments │ │ │ -│ │ └───────────────────────────────────────────────────────────────┘ │ │ -│ └─────────────────────────────────────────────────────────────────────┘ │ -└─────────────────────────────────────────────────────────────────────────────┘ -``` - ---- - -## Class Hierarchy - -``` -StreamSupervisor (interface) - └── SeekableStreamSupervisor - ├── KafkaSupervisor - └── KinesisSupervisor - -SeekableStreamIndexTask - ├── KafkaIndexTask - └── KinesisIndexTask - -RecordSupplier - ├── KafkaRecordSupplier - └── KinesisRecordSupplier -``` - -### Key Classes and Files - -| Class | Location | Purpose | -|---------------------------------|-----------------------------------------------------------------------|--------------------------------------------| -| `SeekableStreamSupervisor` | `indexing-service/.../supervisor/SeekableStreamSupervisor.java` | Abstract base class for stream supervisors | -| `KafkaSupervisor` | `extensions-core/kafka-indexing-service/.../KafkaSupervisor.java` | Kafka-specific supervisor implementation | -| `KinesisSupervisor` | `extensions-core/kinesis-indexing-service/.../KinesisSupervisor.java` | Kinesis-specific supervisor implementation | -| `SeekableStreamIndexTask` | `indexing-service/.../SeekableStreamIndexTask.java` | Abstract base for indexing tasks | -| `SeekableStreamIndexTaskRunner` | `indexing-service/.../SeekableStreamIndexTaskRunner.java` | Task runner with main ingestion loop | -| `RecordSupplier` | `indexing-service/.../common/RecordSupplier.java` | Interface for stream consumer | - ---- - -## Supervisor Lifecycle - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ SUPERVISOR LIFECYCLE │ -└─────────────────────────────────────────────────────────────────────────────┘ - - ┌──────────────┐ - │ start() │ - └──────┬───────┘ - │ - ▼ - ┌──────────────┐ retry (up to 20 times) - │ tryInit() │ ─────────────────────────┐ - └──────┬───────┘ │ - │ success │ - ▼ │ - ┌──────────────────────────────────────┐ │ - │ setupRecordSupplier() │ │ - │ (Creates Kafka/Kinesis consumer) │◄─┘ - └──────┬───────────────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ Start Notice Processing Thread │ - │ (exec.submit -> notices.poll()) │ - └──────┬───────────────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ Schedule Periodic RunNotice │ - │ (scheduledExec.scheduleAtFixedRate) │ - │ Period: ioConfig.period (default 1m)│ - └──────┬───────────────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ Schedule Reporting │ - │ (Lag metrics, offset updates) │ - └──────┬───────────────────────────────┘ - │ - ▼ - ┌──────────────┐ - │ RUNNING │ - └──────────────┘ -``` - -### Key Initialization Code Path -**File:** `SeekableStreamSupervisor.java:1276-1357` - -```java -public void tryInit() { - recordSupplier = setupRecordSupplier(); // Creates Kafka/Kinesis consumer - - exec.submit(() -> { - while (!stopped) { - Notice notice = notices.poll(pollTimeout); - notice.handle(); // Process RunNotice, CheckpointNotice, etc. - } - }); - - scheduledExec.scheduleAtFixedRate( - buildRunTask(), // Adds RunNotice to queue - ioConfig.getStartDelay().getMillis(), - ioConfig.getPeriod().getMillis(), - TimeUnit.MILLISECONDS - ); -} -``` - ---- - -## Main Run Loop Flow - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ runInternal() FLOW │ -│ SeekableStreamSupervisor.java:1733 │ -└─────────────────────────────────────────────────────────────────────────────┘ - - ┌──────────────────────────────────────┐ - │ RunNotice.handle() │ - │ │ │ - │ ▼ │ - │ runInternal() │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 1. possiblyRegisterListener() │ Register TaskRunnerListener - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 2. updatePartitionDataFromStream() │ ◄── PARTITION DISCOVERY - │ - recordSupplier.getPartitionIds()│ - │ - Assign partitions to groups │ - │ - Handle new/expired partitions │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 3. discoverTasks() │ ◄── TASK DISCOVERY - │ - Find existing tasks │ - │ - Create TaskGroup for each │ - │ - Verify checkpoints │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 4. updateTaskStatus() │ - │ - Get startTime from tasks │ - │ - Update task status │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 5. checkTaskDuration() │ ◄── TASK LIFECYCLE - │ - If task exceeded duration │ - │ - Signal tasks to finish │ - │ - Move to pendingCompletion │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 6. checkPendingCompletionTasks() │ - │ - Monitor publishing tasks │ - │ - Handle timeouts/failures │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 7. checkCurrentTaskState() │ - │ - Verify tasks are healthy │ - │ - Kill failed tasks │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 8. checkIfStreamInactiveAndTurn... │ - │ - Idle detection │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 9. createNewTasks() │ ◄── TASK CREATION - │ - Create TaskGroups for groups │ - │ without active tasks │ - │ - Create replica tasks │ - └──────────────────────────────────────┘ -``` - ---- - -## Partition Discovery and Assignment - -### Partition Discovery Flow - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ updatePartitionDataFromStream() │ -│ SeekableStreamSupervisor.java:2849 │ -└─────────────────────────────────────────────────────────────────────────────┘ - - ┌──────────────────────────────────────┐ - │ recordSupplier.getPartitionIds() │ - │ (Kafka: KafkaConsumer.partitionsFor)│ - │ (Kinesis: listShards API) │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ Get stored metadata offsets │ - │ indexerMetadataStorageCoordinator │ - │ .retrieveDataSourceMetadata() │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ Filter partitions: │ - │ - Remove closed partitions │ - │ - Remove expired partitions │ - │ - activePartitionsIdsFromSupplier │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ For each active partition: │ - │ ┌────────────────────────────────┐ │ - │ │ taskGroupId = │ │ - │ │ getTaskGroupIdForPartition() │ │ - │ │ │ │ - │ │ Kafka: partition % taskCount │ │ - │ │ Kinesis: index % taskCount │ │ - │ └────────────────────────────────┘ │ - │ │ - │ partitionGroups.put(taskGroupId, │ - │ partition) │ - │ partitionOffsets.put(partition, │ - │ NOT_SET) │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ If partitions changed: │ - │ - assignRecordSupplierToPartitionIds│ - │ - Set earlyStopTime for │ - │ repartition transition │ - └──────────────────────────────────────┘ -``` - -### Partition Assignment Algorithm - -#### Kafka Partition Assignment -**File:** `KafkaSupervisor.java:141-149` - -```java -@Override -protected int getTaskGroupIdForPartition(KafkaTopicPartition partitionId) { - Integer taskCount = spec.getIoConfig().getTaskCount(); - if (partitionId.isMultiTopicPartition()) { - // Multi-topic: hash based assignment - return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; - } else { - // Single topic: simple modulo - return partitionId.partition() % taskCount; - } -} -``` - -``` -Example: 12 partitions, taskCount = 3 - -Partition 0 → TaskGroup 0 (0 % 3 = 0) -Partition 1 → TaskGroup 1 (1 % 3 = 1) -Partition 2 → TaskGroup 2 (2 % 3 = 2) -Partition 3 → TaskGroup 0 (3 % 3 = 0) -Partition 4 → TaskGroup 1 (4 % 3 = 1) -... -Partition 11 → TaskGroup 2 (11 % 3 = 2) - -TaskGroup 0: [P0, P3, P6, P9] -TaskGroup 1: [P1, P4, P7, P10] -TaskGroup 2: [P2, P5, P8, P11] -``` - -#### Kinesis Shard Assignment -**File:** `KinesisSupervisor.java:213-226` - -```java -@Override -protected int getTaskGroupIdForPartition(String partitionId) { - return getTaskGroupIdForPartitionWithProvidedList(partitionId, partitionIds); -} - -private int getTaskGroupIdForPartitionWithProvidedList(String partitionId, List availablePartitions) { - int index = availablePartitions.indexOf(partitionId); - if (index < 0) { - return index; - } - return availablePartitions.indexOf(partitionId) % spec.getIoConfig().getTaskCount(); -} -``` - -``` -Example: 6 shards, taskCount = 2 - -Shard List (ordered): [shard-000, shard-001, shard-002, shard-003, shard-004, shard-005] - -shard-000 (index 0) → TaskGroup 0 (0 % 2 = 0) -shard-001 (index 1) → TaskGroup 1 (1 % 2 = 1) -shard-002 (index 2) → TaskGroup 0 (2 % 2 = 0) -shard-003 (index 3) → TaskGroup 1 (3 % 2 = 1) -shard-004 (index 4) → TaskGroup 0 (4 % 2 = 0) -shard-005 (index 5) → TaskGroup 1 (5 % 2 = 1) - -TaskGroup 0: [shard-000, shard-002, shard-004] -TaskGroup 1: [shard-001, shard-003, shard-005] -``` - ---- - -## Task Group Management - -### Data Structures - -**File:** `SeekableStreamSupervisor.java:845-865` - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ SUPERVISOR DATA STRUCTURES │ -└─────────────────────────────────────────────────────────────────────────────┘ - -activelyReadingTaskGroups: ConcurrentHashMap -├── TaskGroup 0 -│ ├── groupId: 0 -│ ├── startingSequences: {P0→offset1, P3→offset2, P6→offset3} -│ ├── tasks: ConcurrentHashMap -│ │ ├── "task-abc-123" → TaskData{status, startTime, currentSequences} -│ │ └── "task-abc-456" → TaskData{status, startTime, currentSequences} -│ ├── checkpointSequences: TreeMap> -│ ├── minimumMessageTime, maximumMessageTime -│ ├── exclusiveStartSequenceNumberPartitions -│ └── baseSequenceName: "seq_kafka_topic_abc123..." -├── TaskGroup 1 -│ └── ... -└── TaskGroup 2 - └── ... - -pendingCompletionTaskGroups: ConcurrentHashMap> -├── TaskGroup 0 → [TaskGroup(publishing), TaskGroup(publishing)] -└── ... - -partitionGroups: ConcurrentHashMap> -├── 0 → {P0, P3, P6, P9} -├── 1 → {P1, P4, P7, P10} -└── 2 → {P2, P5, P8, P11} - -partitionOffsets: ConcurrentHashMap -├── P0 → 12345 -├── P1 → 67890 -└── ... - -partitionIds: CopyOnWriteArrayList -└── [P0, P1, P2, P3, P4, P5, P6, P7, P8, P9, P10, P11] -``` - -### TaskGroup Lifecycle - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ TaskGroup STATE MACHINE │ -└─────────────────────────────────────────────────────────────────────────────┘ - - ┌─────────────────────┐ - │ NO TaskGroup │ - │ (partitionGroup │ - │ exists, but no │ - │ TaskGroup yet) │ - └──────────┬──────────┘ - │ createNewTasks() - ▼ - ┌─────────────────────┐ - │ activelyReading │ ◄─────────────────────┐ - │ TaskGroups │ │ - │ │ New task created │ - │ - Tasks are reading │ after previous │ - │ - Ingesting data │ completes │ - └──────────┬──────────┘ │ - │ │ - │ checkTaskDuration(): │ - │ task.startTime + taskDuration │ - │ has passed │ - ▼ │ - ┌─────────────────────┐ │ - │ checkpointTaskGroup │ │ - │ │ │ - │ 1. Pause all tasks │ │ - │ 2. Get highest │ │ - │ offsets │ │ - │ 3. Set end offsets │ │ - │ 4. Resume tasks │ │ - └──────────┬──────────┘ │ - │ │ - ▼ │ - ┌─────────────────────┐ │ - │ pendingCompletion │ │ - │ TaskGroups │ │ - │ │ │ - │ - Tasks are │ │ - │ publishing │ │ - │ - Waiting for │ │ - │ handoff │ │ - └──────────┬──────────┘ │ - │ │ - │ Task SUCCESS │ - ▼ │ - ┌─────────────────────┐ │ - │ TaskGroup removed │───────────────────────┘ - │ from pending │ - │ completion │ - │ │ - │ Offsets updated in │ - │ partitionOffsets │ - └─────────────────────┘ -``` - ---- - -## Task Creation Flow - -**File:** `SeekableStreamSupervisor.java:3854-3975` - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ createNewTasks() FLOW │ -└─────────────────────────────────────────────────────────────────────────────┘ - - ┌──────────────────────────────────────┐ - │ For each groupId in partitionGroups │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ if (!activelyReadingTaskGroups │ Check if TaskGroup - │ .containsKey(groupId)) │ already exists - └──────────────┬───────────────────────┘ - │ No TaskGroup exists - ▼ - ┌──────────────────────────────────────┐ - │ Generate starting sequences: │ - │ generateStartingSequencesFor...() │ - │ │ - │ 1. Check partitionOffsets │ - │ 2. If NOT_SET, check metadata store │ - │ 3. If not in metadata, get from │ - │ stream (earliest/latest) │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ Create new TaskGroup: │ - │ new TaskGroup( │ - │ groupId, │ - │ startingSequences, │ - │ minimumMessageTime, │ - │ maximumMessageTime, │ - │ exclusiveStartSequencePartitions │ - │ ) │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ activelyReadingTaskGroups.put(...) │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ For each TaskGroup: │ - │ if (replicas > tasks.size()) │ - │ createTasksForGroup(groupId, │ - │ replicas - tasks.size()) │ - └──────────────┬───────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────────────────────────────────┐ - │ createTasksForGroup(): │ - │ │ - │ 1. Create SeekableStreamIndexTaskIOConfig │ - │ - startPartitions: group.startingSequences │ - │ - endPartitions: END_OF_PARTITION marker for each partition │ - │ │ - │ 2. createIndexTasks() [Kafka/Kinesis specific] │ - │ - Creates KafkaIndexTask or KinesisIndexTask │ - │ - taskId: random ID with baseSequenceName prefix │ - │ │ - │ 3. taskQueue.add(indexTask) │ - │ - Task is submitted to Overlord's task queue │ - └──────────────────────────────────────────────────────────────────┘ -``` - ---- - -## Task Execution Flow - -### SeekableStreamIndexTaskRunner Main Loop - -**File:** `SeekableStreamIndexTaskRunner.java:399-700` - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ SeekableStreamIndexTaskRunner.run() │ -└─────────────────────────────────────────────────────────────────────────────┘ - - ┌──────────────────────────────────────┐ - │ 1. Initialize │ - │ - Create StreamChunkParser │ - │ - initializeSequences() │ - │ - Register with ChatHandler │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 2. Setup RecordSupplier │ - │ task.newTaskRecordSupplier() │ - │ - KafkaRecordSupplier │ - │ - KinesisRecordSupplier │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 3. Setup Appenderator + Driver │ - │ - newAppenderator() │ - │ - newDriver() │ - │ - driver.startJob() │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 4. Restore/Initialize Offsets │ - │ - currOffsets from metadata │ - │ - or from startSequenceNumbers │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 5. Assign & Seek Partitions │ - │ assignPartitions(recordSupplier) │ - │ seekToStartingSequence() │ - └──────────────┬───────────────────────┘ - │ - ▼ -┌─────────────────────────────────────────────────────────────────────────────┐ -│ MAIN INGESTION LOOP │ -│ │ -│ while (stillReading) { │ -│ │ │ -│ ├─► possiblyPause() // Handle pause/resume from supervisor │ -│ │ │ -│ ├─► if (stopRequested) // Check if supervisor signaled stop │ -│ │ status = PUBLISHING │ -│ │ break │ -│ │ │ -│ ├─► maybePersistAndPublish... // Checkpoint if needed │ -│ │ │ -│ ├─► getRecords(recordSupplier) │ -│ │ └── recordSupplier.poll(timeout) │ -│ │ ├── Kafka: KafkaConsumer.poll() │ -│ │ └── Kinesis: getRecords() API │ -│ │ │ -│ └─► for (record : records) { │ -│ │ │ -│ ├─► verifyRecordInRange() // Check record is in range │ -│ │ │ -│ ├─► parser.parse(record) // Parse → InputRows │ -│ │ │ -│ ├─► driver.add(row, ...) // Add to Appenderator │ -│ │ │ │ -│ │ └─► if (isPushRequired) // Segment is full │ -│ │ driver.persist() │ -│ │ │ -│ └─► Update currOffsets[partition] = record.sequenceNumber │ -│ } │ -│ } │ -│ │ -└─────────────────────────────────────────────────────────────────────────────┘ - │ - ▼ - ┌──────────────────────────────────────┐ - │ 6. Publishing Phase │ - │ status = PUBLISHING │ - │ │ - │ - Persist remaining data │ - │ - Publish segments │ - │ - Update metadata │ - │ - Handoff segments │ - └──────────────────────────────────────┘ -``` - ---- - -## Checkpointing and Publishing - -### Checkpoint Flow - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ CHECKPOINTING MECHANISM │ -└─────────────────────────────────────────────────────────────────────────────┘ - -Supervisor Side (SeekableStreamSupervisor): -━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━ - - ┌──────────────────────────────────────┐ - │ checkTaskDuration() │ - │ (Task has run for taskDuration) │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ checkpointTaskGroup(group, true) │ (finalize=true) - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 1. Pause all tasks: │ - │ taskClient.pauseAsync(taskId) │ - │ → Returns current offsets │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 2. Compute highest offsets │ - │ across all replica tasks │ - │ endOffsets = max(all offsets) │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 3. Set end offsets and finalize: │ - │ taskClient.setEndOffsetsAsync( │ - │ taskId, endOffsets, true) │ - │ → Task will finish and publish │ - └──────────────┬───────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ 4. Move TaskGroup to │ - │ pendingCompletionTaskGroups │ - │ │ - │ 5. Update partitionOffsets with │ - │ endOffsets for next TaskGroup │ - └──────────────────────────────────────┘ - - -Task Side (SeekableStreamIndexTaskRunner): -━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━ - - ┌──────────────────────────────────────┐ - │ POST /pause │ - │ → possiblyPause() │ - │ → status = PAUSED │ - │ → Return currOffsets │ - └──────────────────────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ POST /offsets/end?finish=true │ - │ → setEndOffsets(offsets, true) │ - │ → endOffsets = offsets │ - │ → Resume reading │ - └──────────────────────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ Continue reading until │ - │ currOffset >= endOffset │ - │ for all partitions │ - └──────────────────────────────────────┘ - │ - ┌──────────────▼───────────────────────┐ - │ Transition to PUBLISHING: │ - │ - Persist remaining data │ - │ - Publish segments │ - │ - Checkpoint metadata │ - │ - Handoff to Historical │ - └──────────────────────────────────────┘ -``` - -### Metadata Storage - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ METADATA STORAGE │ -│ (druid_dataSource table) │ -└─────────────────────────────────────────────────────────────────────────────┘ - -dataSource: "my_datasource" -supervisorId: "supervisor_my_datasource" - -commit_metadata_payload: -{ - "type": "kafka", // or "kinesis" - "partitions": { - "type": "end", - "stream": "my_topic", - "partitionSequenceNumberMap": { - "0": 12345, - "1": 67890, - "2": 11111, - ... - } - } -} - -This metadata is used to: -1. Resume from the last committed offset on restart -2. Ensure exactly-once semantics with segment metadata -3. Coordinate between supervisor and tasks -``` - ---- - -## Kafka vs Kinesis Differences - -| Aspect | Kafka | Kinesis | -|--------|-------|---------| -| **Partition ID Type** | `KafkaTopicPartition` (topic + partition int) | `String` (shard ID) | -| **Sequence Type** | `Long` (offset) | `String` (sequence number) | -| **Partition Assignment** | `partition % taskCount` | `indexOf(shard) % taskCount` | -| **Exclusive Start** | Not used (`useExclusiveStartingSequence = false`) | Used (`useExclusiveStartingSequence = true`) | -| **Shard/Partition Expiration** | No (partitions are permanent) | Yes (shards can expire) | -| **Lag Metric** | Record count lag | Time lag (milliseconds) | -| **End of Shard** | N/A | `KinesisSequenceNumber.END_OF_SHARD_MARKER` | -| **RecordSupplier** | `KafkaRecordSupplier` (wraps KafkaConsumer) | `KinesisRecordSupplier` (uses AWS SDK) | - -### Kafka-Specific Features - -**File:** `KafkaSupervisor.java` - -```java -// Multi-topic support with regex patterns -private final Pattern pattern; // For topic regex matching - -// Partition assignment for multi-topic -if (partitionId.isMultiTopicPartition()) { - return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; -} - -// Record lag (offset-based) -@Override -protected Map getPartitionRecordLag() { - return latestSequenceFromStream - currentOffsets; -} -``` - -### Kinesis-Specific Features - -**File:** `KinesisSupervisor.java` - -```java -// Shard expiration handling -@Override -protected boolean supportsPartitionExpiration() { - return true; -} - -@Override -protected boolean isEndOfShard(String seqNum) { - return KinesisSequenceNumber.END_OF_SHARD_MARKER.equals(seqNum); -} - -// Shard rebalancing on expiration -@Override -protected Map> recomputePartitionGroupsForExpiration(Set availablePartitions) { - // Redistribute shards evenly across task groups - List availablePartitionsList = new ArrayList<>(availablePartitions); - for (String partition : availablePartitions) { - int newTaskGroupId = indexOf(partition) % taskCount; - newPartitionGroups.get(newTaskGroupId).add(partition); - } - return newPartitionGroups; -} - -// Time-based lag (no record count available) -@Override -protected Map getPartitionTimeLag() { - return currentPartitionTimeLag; // milliseconds behind latest -} -``` - ---- - -## Key Configuration Parameters - -### Supervisor IOConfig - -| Parameter | Default | Description | -|-----------|---------|-------------| -| `taskCount` | 1 | Number of task groups (parallel readers) | -| `replicas` | 1 | Number of replica tasks per group | -| `taskDuration` | PT1H | How long each task runs before publishing | -| `completionTimeout` | PT30M | Timeout for task completion | -| `startDelay` | PT5S | Delay before first run | -| `period` | PT30S | How often supervisor runs | -| `useEarliestSequenceNumber` | false | Start from earliest or latest | -| `maxAllowedStops` | 1 | Maximum concurrent task group stops | - -### Supervisor TuningConfig - -| Parameter | Default | Description | -|-----------|---------|-------------| -| `workerThreads` | null | Thread pool size for task communication | -| `chatThreads` | null | Threads for HTTP endpoints | -| `chatRetries` | 8 | Retries for task communication | -| `httpTimeout` | PT10S | HTTP request timeout | -| `shutdownTimeout` | PT80S | Shutdown wait time | -| `repartitionTransitionDuration` | PT2M | Cooldown after partition changes | - ---- - -## Summary Diagram - -``` -┌─────────────────────────────────────────────────────────────────────────────┐ -│ COMPLETE STREAM INGESTION FLOW │ -└─────────────────────────────────────────────────────────────────────────────┘ - -Stream (Kafka/Kinesis) Overlord MiddleManager/Peon -━━━━━━━━━━━━━━━━━━━━━━━━━ ━━━━━━━━━━━━━━━━━━━━━━━━ ━━━━━━━━━━━━━━━━━━━━━━━ - -┌───────────────────┐ -│ Topic/Stream │ -│ ┌───┬───┬───┬───┐ │ ┌──────────────────────┐ -│ │P0 │P1 │P2 │P3 │ │─────►│ SeekableStream │ -│ └───┴───┴───┴───┘ │ │ Supervisor │ -└───────────────────┘ │ │ - │ 1. getPartitionIds() │ - │ 2. Assign to groups │ - │ 3. Create TaskGroups │ - └──────────┬───────────┘ - │ - ▼ - ┌──────────────────────┐ - │ createIndexTasks() │ - │ │ - │ TaskGroup 0: │ - │ Task-0a (replica) │────────┐ - │ Task-0b (replica) │────────┤ - │ │ │ - │ TaskGroup 1: │ │ - │ Task-1a (replica) │────────┤ - └──────────────────────┘ │ - ▼ - ┌─────────────────────┐ - │ SeekableStream │ - │ IndexTaskRunner │ - │ │ - │ while(reading) { │ -┌───────────────────┐ │ poll(stream) │ -│ RecordSupplier │◄─────────────────────────│ parse(records) │ -│ (poll records) │ │ add(appenderator) │ -└───────────────────┘ │ } │ - │ │ - │ publish segments │──────► Deep Storage - │ update metadata │──────► Metadata Store - │ handoff │──────► Historical - └─────────────────────┘ -``` - ---- - ---- - 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 34b3967ccedb..0bb5489548dd 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 @@ -3354,43 +3354,43 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing activelyReadingTaskGroups.entrySet().stream().sorted( - Comparator.comparingLong( - taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() - ) - ) - .forEach(entry -> { - Integer groupId = entry.getKey(); - TaskGroup group = entry.getValue(); - - final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); - final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); - if (stopTasksEarly || group.getHandoffEarly()) { - // If handoffEarly has been set, stop tasks irrespective of stopTaskCount - log.info( - "Stopping taskGroup[%d] early after running for duration[%s].", - groupId, runDuration - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - if (group.getHandoffEarly()) { - numStoppedTasks.getAndIncrement(); - } - } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { - // Stop this task group if it has run longer than the configured duration - // and the pending task groups are less than the configured stop task count. - int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() - .mapToInt(List::size).sum(); - if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { - log.info( - "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", - groupId, runDuration, ioConfig.getTaskDuration() - ); - futureGroupIds.add(groupId); - futures.add(checkpointTaskGroup(group, true)); - numStoppedTasks.getAndIncrement(); - } - } - }); + Comparator.comparingLong( + taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() + ) + ) + .forEach(entry -> { + Integer groupId = entry.getKey(); + TaskGroup group = entry.getValue(); + + final DateTime earliestTaskStart = computeEarliestTaskStartTime(group); + final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis()); + if (stopTasksEarly || group.getHandoffEarly()) { + // If handoffEarly has been set, stop tasks irrespective of stopTaskCount + log.info( + "Stopping taskGroup[%d] early after running for duration[%s].", + groupId, runDuration + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + if (group.getHandoffEarly()) { + numStoppedTasks.getAndIncrement(); + } + } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) { + // Stop this task group if it has run longer than the configured duration + // and the pending task groups are less than the configured stop task count. + int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream() + .mapToInt(List::size).sum(); + if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) { + log.info( + "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].", + groupId, runDuration, ioConfig.getTaskDuration() + ); + futureGroupIds.add(groupId); + futures.add(checkpointTaskGroup(group, true)); + numStoppedTasks.getAndIncrement(); + } + } + }); List>> results = coalesceAndAwait(futures); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); From 24fdd708d774967cb12e09fed2a5abfcccd05a9e Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 16:58:42 +0200 Subject: [PATCH 15/20] Fix the SupervisorManagerTest since addition of Supervisor#createAutoscaler interface method --- .../supervisor/SupervisorManagerTest.java | 45 ++++++++++++++----- 1 file changed, 33 insertions(+), 12 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index b61c93af3c8f..df1e4d37aa89 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -102,7 +102,9 @@ public void testCreateUpdateAndRemoveSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); metadataSupervisorManager.insert("id1", spec); supervisor3.start(); + EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -115,6 +117,7 @@ public void testCreateUpdateAndRemoveSupervisor() resetAll(); supervisor2.start(); + EasyMock.expect(supervisor2.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.stop(true); replayAll(); @@ -164,6 +167,7 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(ImmutableMap.of()); metadataSupervisorManager.insert("id1", spec); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -225,6 +229,7 @@ public void testShouldUpdateSupervisor() ); EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); replayAll(); manager.start(); Assert.assertFalse(manager.shouldUpdateSupervisor(spec)); @@ -249,6 +254,7 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept ); EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); exception.expect(DruidException.class); replayAll(); manager.start(); @@ -332,6 +338,7 @@ public void testGetSupervisorStatus() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); EasyMock.expect(supervisor1.getStatus()).andReturn(report); replayAll(); @@ -352,6 +359,7 @@ public void testHandoffTaskGroupsEarly() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.handoffTaskGroupsEarly(ImmutableList.of(1)); replayAll(); @@ -373,6 +381,7 @@ public void testHandoffTaskGroupsEarlyOnNonStreamSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor3.start(); + EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); replayAll(); @@ -414,6 +423,8 @@ public void testStartIndividualSupervisorsFailStart() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor3.start(); + EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.start(); EasyMock.expectLastCall().andThrow(new RuntimeException("supervisor explosion")); replayAll(); @@ -433,7 +444,9 @@ public void testNoPersistOnFailedStart() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(Collections.emptyMap()); metadataSupervisorManager.insert(EasyMock.eq("id1"), EasyMock.capture(capturedInsert)); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.stop(true); + EasyMock.expect(supervisor2.createAutoscaler()).andReturn(null).anyTimes(); supervisor2.start(); EasyMock.expectLastCall().andThrow(new RuntimeException("supervisor failed to start")); replayAll(); @@ -461,6 +474,7 @@ public void testStopThrowsException() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.stopAsync(); EasyMock.expectLastCall().andThrow(new RuntimeException("RTE")); replayAll(); @@ -479,6 +493,7 @@ public void testResetSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.reset(EasyMock.anyObject(DataSourceMetadata.class)); replayAll(); @@ -498,6 +513,7 @@ public void testResetOnNonStreamSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor3.start(); + EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -533,6 +549,7 @@ public void testResetSupervisorWithSpecificOffsets() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.resetOffsets(datasourceMetadata); replayAll(); @@ -558,7 +575,9 @@ public void testCreateSuspendResumeAndStopSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); metadataSupervisorManager.insert("id1", spec); supervisor3.start(); + EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -574,6 +593,7 @@ public void testCreateSuspendResumeAndStopSupervisor() resetAll(); metadataSupervisorManager.insert(EasyMock.eq("id1"), EasyMock.capture(capturedInsert)); supervisor2.start(); + EasyMock.expect(supervisor2.createAutoscaler()).andReturn(null).anyTimes(); supervisor1.stop(true); replayAll(); @@ -589,6 +609,7 @@ public void testCreateSuspendResumeAndStopSupervisor() metadataSupervisorManager.insert(EasyMock.eq("id1"), EasyMock.capture(capturedInsert)); supervisor2.stop(true); supervisor1.start(); + EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); replayAll(); manager.suspendOrResumeSupervisor("id1", false); @@ -631,29 +652,29 @@ public void testGetActiveSupervisorIdForDatasourceWithAppendLock() metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); SeekableStreamSupervisorSpec suspendedSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); - Supervisor suspendedSupervisor = EasyMock.mock(SeekableStreamSupervisor.class); + Supervisor suspendedSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); EasyMock.expect(suspendedSpec.getId()).andReturn("suspendedSpec").anyTimes(); EasyMock.expect(suspendedSpec.isSuspended()).andReturn(true).anyTimes(); EasyMock.expect(suspendedSpec.getDataSources()).andReturn(ImmutableList.of("suspendedDS")).anyTimes(); EasyMock.expect(suspendedSpec.createSupervisor()).andReturn(suspendedSupervisor).anyTimes(); EasyMock.expect(suspendedSpec.createAutoscaler(suspendedSupervisor)).andReturn(null).anyTimes(); EasyMock.expect(suspendedSpec.getContext()).andReturn(null).anyTimes(); - EasyMock.replay(suspendedSpec); + EasyMock.replay(suspendedSpec, suspendedSupervisor); metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); SeekableStreamSupervisorSpec activeSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); - Supervisor activeSupervisor = EasyMock.mock(SeekableStreamSupervisor.class); + Supervisor activeSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); EasyMock.expect(activeSpec.getId()).andReturn("activeSpec").anyTimes(); EasyMock.expect(activeSpec.isSuspended()).andReturn(false).anyTimes(); EasyMock.expect(activeSpec.getDataSources()).andReturn(ImmutableList.of("activeDS")).anyTimes(); EasyMock.expect(activeSpec.createSupervisor()).andReturn(activeSupervisor).anyTimes(); EasyMock.expect(activeSpec.createAutoscaler(activeSupervisor)).andReturn(null).anyTimes(); EasyMock.expect(activeSpec.getContext()).andReturn(null).anyTimes(); - EasyMock.replay(activeSpec); + EasyMock.replay(activeSpec, activeSupervisor); metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); SeekableStreamSupervisorSpec activeSpecWithConcurrentLocks = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); - Supervisor activeSupervisorWithConcurrentLocks = EasyMock.mock(SeekableStreamSupervisor.class); + Supervisor activeSupervisorWithConcurrentLocks = EasyMock.createNiceMock(SeekableStreamSupervisor.class); EasyMock.expect(activeSpecWithConcurrentLocks.getId()).andReturn("activeSpecWithConcurrentLocks").anyTimes(); EasyMock.expect(activeSpecWithConcurrentLocks.isSuspended()).andReturn(false).anyTimes(); EasyMock.expect(activeSpecWithConcurrentLocks.getDataSources()) @@ -664,11 +685,11 @@ public void testGetActiveSupervisorIdForDatasourceWithAppendLock() .andReturn(null).anyTimes(); EasyMock.expect(activeSpecWithConcurrentLocks.getContext()) .andReturn(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)).anyTimes(); - EasyMock.replay(activeSpecWithConcurrentLocks); + EasyMock.replay(activeSpecWithConcurrentLocks, activeSupervisorWithConcurrentLocks); metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); SeekableStreamSupervisorSpec activeAppendSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); - Supervisor activeAppendSupervisor = EasyMock.mock(SeekableStreamSupervisor.class); + Supervisor activeAppendSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); EasyMock.expect(activeAppendSpec.getId()).andReturn("activeAppendSpec").anyTimes(); EasyMock.expect(activeAppendSpec.isSuspended()).andReturn(false).anyTimes(); EasyMock.expect(activeAppendSpec.getDataSources()).andReturn(ImmutableList.of("activeAppendDS")).anyTimes(); @@ -678,12 +699,12 @@ public void testGetActiveSupervisorIdForDatasourceWithAppendLock() Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND.name() )).anyTimes(); - EasyMock.replay(activeAppendSpec); + EasyMock.replay(activeAppendSpec, activeAppendSupervisor); metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); // A supervisor with useConcurrentLocks set to false explicitly must not use an append lock SeekableStreamSupervisorSpec specWithUseConcurrentLocksFalse = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); - Supervisor supervisorWithUseConcurrentLocksFalse = EasyMock.mock(SeekableStreamSupervisor.class); + Supervisor supervisorWithUseConcurrentLocksFalse = EasyMock.createNiceMock(SeekableStreamSupervisor.class); EasyMock.expect(specWithUseConcurrentLocksFalse.getId()).andReturn("useConcurrentLocksFalse").anyTimes(); EasyMock.expect(specWithUseConcurrentLocksFalse.isSuspended()).andReturn(false).anyTimes(); EasyMock.expect(specWithUseConcurrentLocksFalse.getDataSources()) @@ -697,7 +718,7 @@ public void testGetActiveSupervisorIdForDatasourceWithAppendLock() Tasks.TASK_LOCK_TYPE, TaskLockType.APPEND.name() )).anyTimes(); - EasyMock.replay(specWithUseConcurrentLocksFalse); + EasyMock.replay(specWithUseConcurrentLocksFalse, supervisorWithUseConcurrentLocksFalse); metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); replayAll(); @@ -737,7 +758,7 @@ public void testRegisterUpgradedPendingSegmentOnSupervisor() metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); SeekableStreamSupervisorSpec streamingSpec = EasyMock.createNiceMock(SeekableStreamSupervisorSpec.class); - SeekableStreamSupervisor streamSupervisor = EasyMock.mock(SeekableStreamSupervisor.class); + SeekableStreamSupervisor streamSupervisor = EasyMock.createNiceMock(SeekableStreamSupervisor.class); streamSupervisor.registerNewVersionOfPendingSegment(EasyMock.anyObject()); EasyMock.expectLastCall().once(); EasyMock.expect(streamingSpec.getId()).andReturn("sss").anyTimes(); @@ -746,7 +767,7 @@ public void testRegisterUpgradedPendingSegmentOnSupervisor() EasyMock.expect(streamingSpec.createSupervisor()).andReturn(streamSupervisor).anyTimes(); EasyMock.expect(streamingSpec.createAutoscaler(streamSupervisor)).andReturn(null).anyTimes(); EasyMock.expect(streamingSpec.getContext()).andReturn(null).anyTimes(); - EasyMock.replay(streamingSpec); + EasyMock.replay(streamingSpec, streamSupervisor); metadataSupervisorManager.insert(EasyMock.anyString(), EasyMock.anyObject()); EasyMock.expectLastCall().once(); From 833cb881ea31472e6a4c5250722447b02e93ad44 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Tue, 23 Dec 2025 21:42:40 +0200 Subject: [PATCH 16/20] Increase test coverage --- .../SeekableStreamSupervisorSpecTest.java | 203 ++++++++ .../autoscaler/CostBasedAutoScalerTest.java | 470 +++++++++++++++++- 2 files changed, 671 insertions(+), 2 deletions(-) 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 58ffb8438e0d..f5b925e2c59d 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 @@ -37,6 +37,7 @@ import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; @@ -62,6 +63,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.TestSupervisorSpec; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.aggregation.CountAggregatorFactory; @@ -678,6 +680,103 @@ public void testAutoScalerCreated() Assert.assertTrue(autoscaler4 instanceof NoopTaskAutoScaler); } + @Test + public void testAutoScalerReturnsNoopWhenSupervisorIsNotSeekableStreamSupervisor() + { + // Test the branch where supervisor instanceof SeekableStreamSupervisor is false + HashMap autoScalerConfig = new HashMap<>(); + autoScalerConfig.put("enableTaskAutoScaler", true); + autoScalerConfig.put("taskCountMax", 8); + autoScalerConfig.put("taskCountMin", 1); + + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + EasyMock.expect(seekableStreamSupervisorIOConfig.getAutoScalerConfig()) + .andReturn(mapper.convertValue(autoScalerConfig, AutoScalerConfig.class)) + .anyTimes(); + EasyMock.expect(seekableStreamSupervisorIOConfig.getStream()).andReturn("stream").anyTimes(); + EasyMock.replay(seekableStreamSupervisorIOConfig); + + EasyMock.expect(supervisor4.getActiveTaskGroupsCount()).andReturn(0).anyTimes(); + EasyMock.expect(supervisor4.getIoConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.replay(supervisor4); + + TestSeekableStreamSupervisorSpec spec = new TestSeekableStreamSupervisorSpec( + ingestionSchema, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig, + supervisor4, + "id1" + ); + + // Create a non-SeekableStreamSupervisor mock + Supervisor nonSeekableStreamSupervisor = EasyMock.mock(Supervisor.class); + EasyMock.replay(nonSeekableStreamSupervisor); + + // When passing a non-SeekableStreamSupervisor, should return NoopTaskAutoScaler + SupervisorTaskAutoScaler autoscaler = spec.createAutoscaler(nonSeekableStreamSupervisor); + Assert.assertTrue( + "Should return NoopTaskAutoScaler when supervisor is not SeekableStreamSupervisor", + autoscaler instanceof NoopTaskAutoScaler + ); + } + + @Test + public void testAutoScalerReturnsNoopWhenConfigIsNull() + { + // Test the branch where autoScalerConfig is null + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + EasyMock.expect(seekableStreamSupervisorIOConfig.getAutoScalerConfig()) + .andReturn(null) + .anyTimes(); + EasyMock.expect(seekableStreamSupervisorIOConfig.getStream()).andReturn("stream").anyTimes(); + EasyMock.replay(seekableStreamSupervisorIOConfig); + + EasyMock.expect(supervisor4.getActiveTaskGroupsCount()).andReturn(0).anyTimes(); + EasyMock.expect(supervisor4.getIoConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.replay(supervisor4); + + TestSeekableStreamSupervisorSpec spec = new TestSeekableStreamSupervisorSpec( + ingestionSchema, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig, + supervisor4, + "id1" + ); + + // When autoScalerConfig is null, should return NoopTaskAutoScaler + SupervisorTaskAutoScaler autoscaler = spec.createAutoscaler(supervisor4); + Assert.assertTrue( + "Should return NoopTaskAutoScaler when autoScalerConfig is null", + autoscaler instanceof NoopTaskAutoScaler + ); + } + @Test public void testDefaultAutoScalerConfigCreatedWithDefault() { @@ -972,6 +1071,110 @@ public void testSeekableStreamSupervisorSpecWithNoScalingOnIdleSupervisor() thro autoScaler.stop(); } + @Test + public void test_dynamicAllocationNotice_skips_whenSupervisorSuspended() throws InterruptedException + { + EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig(1, true)).anyTimes(); + EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); + EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + // Suspended → DynamicAllocationTasksNotice should return early and not scale + EasyMock.expect(spec.isSuspended()).andReturn(true).anyTimes(); + EasyMock.replay(spec); + + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.replay(taskMaster); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); + LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( + supervisor, + DATASOURCE, + mapper.convertValue( + getScaleOutProperties(2), + LagBasedAutoScalerConfig.class + ), + spec, + emitter + ); + + supervisor.start(); + autoScaler.start(); + supervisor.runInternal(); + + int before = supervisor.getIoConfig().getTaskCount(); + Thread.sleep(1000); + int after = supervisor.getIoConfig().getTaskCount(); + // No scaling expected because supervisor is suspended + Assert.assertEquals(before, after); + + autoScaler.reset(); + autoScaler.stop(); + } + + @Test + public void test_changeTaskCountInIOConfig_handlesExceptionAndStillUpdatesTaskCount() throws InterruptedException + { + EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(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.replay(spec); + + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + // SupervisorManager present but metadata insert fails → should be handled + SupervisorManager sm = EasyMock.createMock(SupervisorManager.class); + MetadataSupervisorManager msm = EasyMock.createMock(MetadataSupervisorManager.class); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(sm)).anyTimes(); + EasyMock.expect(sm.getMetadataSupervisorManager()).andReturn(msm).anyTimes(); + msm.insert(EasyMock.anyString(), EasyMock.anyObject()); + EasyMock.expectLastCall().andThrow(new RuntimeException("boom")).anyTimes(); + EasyMock.replay(taskMaster, sm, msm); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(10); + LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( + supervisor, + DATASOURCE, + mapper.convertValue( + getScaleOutProperties(2), + LagBasedAutoScalerConfig.class + ), + spec, + emitter + ); + + supervisor.start(); + autoScaler.start(); + supervisor.runInternal(); + + int before = supervisor.getIoConfig().getTaskCount(); + Assert.assertEquals(1, before); + Thread.sleep(1000); // allow one dynamic allocation cycle + int after = supervisor.getIoConfig().getTaskCount(); + // Even though metadata insert failed, taskCount should still be updated in ioConfig + Assert.assertEquals(2, after); + + autoScaler.reset(); + autoScaler.stop(); + } + @Test public void testSeekableStreamSupervisorSpecWithScaleOutSmallPartitionNumber() throws InterruptedException { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java index 977c30a8ff0c..0fc596e1bff4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.seekablestream.supervisor.autoscaler; -import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; @@ -35,6 +34,9 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters.FIFTEEN_MINUTE_NAME; +import static org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters.FIVE_MINUTE_NAME; +import static org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters.ONE_MINUTE_NAME; import static org.mockito.Mockito.when; public class CostBasedAutoScalerTest @@ -156,6 +158,364 @@ public void testExtractProcessingRateMovingAverage() Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(validStats), 0.0001); } + @Test + public void testExtractMovingAverageFifteenMinuteFallback() + { + // Test that 15-minute average is preferred when available + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + group.put("task-0", buildTaskStatsWithMovingAverageForInterval(FIFTEEN_MINUTE_NAME, 1500.0)); + stats.put("0", group); + Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageOneMinuteFallback() + { + // Test that 1-minute average is used as final fallback + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + group.put( + "task-0", + buildTaskStatsWithMovingAverageForInterval(ONE_MINUTE_NAME, 500.0) + ); + stats.put("0", group); + Assert.assertEquals(500.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAveragePrefersFifteenOverFive() + { + // Test that 15-minute average is preferred over 5-minute when both are available + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + group.put("task-0", buildTaskStatsWithMultipleMovingAverages(1500.0, 1000.0, 500.0)); + stats.put("0", group); + // Should use 15-minute average (1500.0), not 5-minute (1000.0) or 1-minute (500.0) + Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testComputeTaskCountForScaleActionScaleUp() + { + // Test scale-up scenario: optimal > current + // With low idle ratio and high lag, should want to scale up + CostMetrics highLagMetrics = new CostMetrics( + 10000.0, // high lag + 5, // current task count + 100, // partition count + 0.1, // low idle ratio (busy) + 3600, + 1000.0 + ); + + int result = autoScaler.computeOptimalTaskCount(highLagMetrics); + // The algorithm should evaluate different task counts + Assert.assertTrue("Should return a valid task count", result >= -1); + } + + @Test + public void testComputeTaskCountForScaleActionNoScale() + { + // Test no-scale scenario: optimal == current + CostMetrics balancedMetrics = new CostMetrics( + 100.0, // moderate lag + 25, // current task count + 100, // partition count + 0.4, // moderate idle ratio + 3600, + 1000.0 + ); + + int result = autoScaler.computeOptimalTaskCount(balancedMetrics); + // Either returns -1 (no change) or a different task count + Assert.assertTrue("Result should be -1 or a valid positive number", result >= -1); + } + + @Test + public void testComputeOptimalTaskCountWithNegativePartitions() + { + CostMetrics invalidMetrics = new CostMetrics( + 100.0, + 10, + -5, // negative partition count + 0.3, + 3600, + 1000.0 + ); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(invalidMetrics)); + } + + @Test + public void testComputeOptimalTaskCountWithNegativeTaskCount() + { + CostMetrics invalidMetrics = new CostMetrics( + 100.0, + -1, // negative task count + 100, + 0.3, + 3600, + 1000.0 + ); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(invalidMetrics)); + } + + @Test + public void testComputeValidTaskCountsWithSinglePartition() + { + // Edge case: single partition + int[] validTaskCounts = CostBasedAutoScaler.computeValidTaskCounts(1, 1); + Assert.assertTrue("Should have at least one valid count", validTaskCounts.length > 0); + Assert.assertTrue("Should contain 1 as valid count", contains(validTaskCounts, 1)); + } + + @Test + public void testComputeValidTaskCountsWithNegativePartitions() + { + // Negative partitions should return empty array + int[] validTaskCounts = CostBasedAutoScaler.computeValidTaskCounts(-5, 10); + Assert.assertEquals(0, validTaskCounts.length); + } + + @Test + public void testExtractPollIdleRatioWithNonMapTaskMetric() + { + // Test branch where taskMetric is not a Map + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + group.put("task-0", "not-a-map"); + stats.put("0", group); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); + } + + @Test + public void testExtractPollIdleRatioWithMissingAutoscalerMetrics() + { + // Test branch where autoscaler metrics map is present but poll idle ratio is missing + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map emptyAutoscalerMetrics = new HashMap<>(); + taskStats.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, emptyAutoscalerMetrics); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageWithNonMapTaskMetric() + { + // Test branch where taskMetric is not a Map + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + group.put("task-0", "not-a-map"); + stats.put("0", group); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageWithMissingBuildSegments() + { + // Test branch where movingAverages exists but buildSegments is missing + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map movingAverages = new HashMap<>(); + taskStats.put("movingAverages", movingAverages); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageWithNonMapMovingAverage() + { + // Test branch where movingAveragesObj is not a Map + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + taskStats.put("movingAverages", "not-a-map"); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testComputeTaskCountForScaleActionReturnsMinusOneWhenScaleDown() + { + // When optimal < current, computeTaskCountForScaleAction should return -1 + // This tests the ternary: optimalTaskCount >= currentTaskCount ? optimalTaskCount : -1 + // Create a scenario where the algorithm wants to scale down (high idle ratio) + CostMetrics highIdleMetrics = new CostMetrics( + 10.0, // low lag + 50, // current task count (high) + 100, // partition count + 0.9, // very high idle ratio (underutilized) + 3600, + 1000.0 + ); + + // computeOptimalTaskCount may return a lower task count + int optimalResult = autoScaler.computeOptimalTaskCount(highIdleMetrics); + // The test verifies that computeTaskCountForScaleAction handles scale-down correctly + Assert.assertTrue("Scale down scenario should return optimal <= current", optimalResult <= 50); + } + + @Test + public void testComputeTaskCountForScaleActionReturnsPositiveWhenScaleUp() + { + // When optimal > current, computeTaskCountForScaleAction should return the optimal value + // Create a scenario with low idle (tasks are busy) and some lag + CostMetrics busyMetrics = new CostMetrics( + 5000.0, // significant lag + 5, // low current task count + 100, // partition count (20 partitions per task) + 0.05, // very low idle ratio (tasks are very busy) + 3600, + 1000.0 + ); + + int optimalResult = autoScaler.computeOptimalTaskCount(busyMetrics); + // With very low idle ratio, algorithm should evaluate higher task counts + Assert.assertTrue("Busy scenario result should be valid", optimalResult >= -1); + } + + @Test + public void testComputeOptimalTaskCountWhenOptimalEqualsCurrent() + { + // Test the branch where optimalTaskCount == currentTaskCount returns -1 + // Create balanced metrics that likely result in current count being optimal + CostMetrics balancedMetrics = new CostMetrics( + 50.0, // low lag + 20, // current task count + 100, // partition count (5 partitions per task) + 0.5, // moderate idle ratio + 3600, + 1000.0 + ); + + int result = autoScaler.computeOptimalTaskCount(balancedMetrics); + // Either -1 (optimal == current) or a different task count + Assert.assertTrue("Result should be -1 or positive", result >= -1); + } + + @Test + public void testExtractPollIdleRatioWithNonMapAutoscalerMetrics() + { + // Test branch where AUTOSCALER_METRICS_KEY exists but is not a Map + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + taskStats.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, "not-a-map"); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); + } + + @Test + public void testExtractPollIdleRatioWithNonNumberPollIdleRatio() + { + // Test branch where pollIdleRatioAvg exists but is not a Number + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map autoscalerMetrics = new HashMap<>(); + autoscalerMetrics.put(SeekableStreamIndexTaskRunner.POLL_IDLE_RATIO_KEY, "not-a-number"); + taskStats.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, autoscalerMetrics); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageWithNonMapBuildSegments() + { + // Test branch where buildSegmentsObj is not a Map + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map movingAverages = new HashMap<>(); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, "not-a-map"); + taskStats.put("movingAverages", movingAverages); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageWithNonMapIntervalData() + { + // Test branch where the 15min/5min/1min interval data is not a Map + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map movingAverages = new HashMap<>(); + Map buildSegments = new HashMap<>(); + buildSegments.put(FIFTEEN_MINUTE_NAME, "not-a-map"); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); + taskStats.put("movingAverages", movingAverages); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageWithNonNumberProcessedRate() + { + // Test branch where processedRate is not a Number + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map movingAverages = new HashMap<>(); + Map buildSegments = new HashMap<>(); + Map fifteenMin = new HashMap<>(); + fifteenMin.put(RowIngestionMeters.PROCESSED, "not-a-number"); + buildSegments.put(FIFTEEN_MINUTE_NAME, fifteenMin); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); + taskStats.put("movingAverages", movingAverages); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageFallsBackToFiveMinuteWhenFifteenMinuteNull() + { + // Test the fallback from 15min to 5min when 15min is explicitly null + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map movingAverages = new HashMap<>(); + Map buildSegments = new HashMap<>(); + // Explicitly set 15min to null (not just missing) + buildSegments.put(FIFTEEN_MINUTE_NAME, null); + buildSegments.put(FIVE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, 750.0)); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); + taskStats.put("movingAverages", movingAverages); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(750.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + + @Test + public void testExtractMovingAverageFallsBackToOneMinuteWhenBothNull() + { + // Test the fallback from 15min to 5min to 1min when both 15min and 5min are null + Map> stats = new HashMap<>(); + Map group = new HashMap<>(); + Map taskStats = new HashMap<>(); + Map movingAverages = new HashMap<>(); + Map buildSegments = new HashMap<>(); + buildSegments.put(FIFTEEN_MINUTE_NAME, null); + buildSegments.put(FIVE_MINUTE_NAME, null); + buildSegments.put(ONE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, 250.0)); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); + taskStats.put("movingAverages", movingAverages); + group.put("task-0", taskStats); + stats.put("0", group); + Assert.assertEquals(250.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + } + private CostMetrics createMetrics( double avgPartitionLag, int currentTaskCount, @@ -196,7 +556,10 @@ private Map buildTaskStatsWithPollIdle(double pollIdleRatio) private Map buildTaskStatsWithMovingAverage(double processedRate) { Map buildSegments = new HashMap<>(); - buildSegments.put(DropwizardRowIngestionMeters.FIVE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, processedRate)); + buildSegments.put( + FIVE_MINUTE_NAME, + Map.of(RowIngestionMeters.PROCESSED, processedRate) + ); Map movingAverages = new HashMap<>(); movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); @@ -205,4 +568,107 @@ private Map buildTaskStatsWithMovingAverage(double processedRate taskStats.put("movingAverages", movingAverages); return taskStats; } + + private Map buildTaskStatsWithMovingAverageForInterval(String intervalName, double processedRate) + { + Map buildSegments = new HashMap<>(); + buildSegments.put(intervalName, Map.of(RowIngestionMeters.PROCESSED, processedRate)); + + Map movingAverages = new HashMap<>(); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); + + Map taskStats = new HashMap<>(); + taskStats.put("movingAverages", movingAverages); + return taskStats; + } + + private Map buildTaskStatsWithMultipleMovingAverages( + double fifteenMinRate, + double fiveMinRate, + double oneMinRate + ) + { + Map buildSegments = new HashMap<>(); + buildSegments.put( + FIFTEEN_MINUTE_NAME, + Map.of(RowIngestionMeters.PROCESSED, fifteenMinRate) + ); + buildSegments.put(FIVE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, fiveMinRate)); + buildSegments.put(ONE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, oneMinRate)); + + Map movingAverages = new HashMap<>(); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); + + Map taskStats = new HashMap<>(); + taskStats.put("movingAverages", movingAverages); + return taskStats; + } + + @Test + public void testComputeValidTaskCountsWhenCurrentExceedsPartitions() + { + // the currentTaskCount > partitionCount should still yield valid, + // deduplicated options + int[] counts = CostBasedAutoScaler.computeValidTaskCounts(2, 5); + Assert.assertEquals(2, counts.length); + Assert.assertTrue(contains(counts, 1)); + Assert.assertTrue(contains(counts, 2)); + } + + @Test + public void testComputeTaskCountForRolloverReturnsMinusOneWhenSuspended() + { + // Arrange: build autoscaler with suspended spec so collectMetrics returns null + SupervisorSpec spec = Mockito.mock(SupervisorSpec.class); + SeekableStreamSupervisor supervisor = Mockito.mock(SeekableStreamSupervisor.class); + ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + SeekableStreamSupervisorIOConfig ioConfig = Mockito.mock(SeekableStreamSupervisorIOConfig.class); + + when(spec.getId()).thenReturn("s-up"); + when(spec.isSuspended()).thenReturn(true); + when(supervisor.getIoConfig()).thenReturn(ioConfig); + when(ioConfig.getStream()).thenReturn("stream"); + + CostBasedAutoScalerConfig cfg = CostBasedAutoScalerConfig.builder() + .taskCountMax(10) + .taskCountMin(1) + .enableTaskAutoScaler(true) + .lagWeight(0.5) + .idleWeight(0.5) + .build(); + + CostBasedAutoScaler scaler = new CostBasedAutoScaler(supervisor, cfg, spec, emitter); + + // Then + Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); + } + + @Test + public void testComputeTaskCountForRolloverReturnsMinusOneWhenLagStatsNull() + { + // Arrange: collectMetrics should early-return when lagStats is null + SupervisorSpec spec = Mockito.mock(SupervisorSpec.class); + SeekableStreamSupervisor supervisor = Mockito.mock(SeekableStreamSupervisor.class); + ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + SeekableStreamSupervisorIOConfig ioConfig = Mockito.mock(SeekableStreamSupervisorIOConfig.class); + + when(spec.getId()).thenReturn("s-up"); + when(spec.isSuspended()).thenReturn(false); + when(supervisor.computeLagStats()).thenReturn(null); + when(supervisor.getIoConfig()).thenReturn(ioConfig); + when(ioConfig.getStream()).thenReturn("stream"); + + CostBasedAutoScalerConfig cfg = CostBasedAutoScalerConfig.builder() + .taskCountMax(10) + .taskCountMin(1) + .enableTaskAutoScaler(true) + .lagWeight(0.5) + .idleWeight(0.5) + .build(); + + CostBasedAutoScaler scaler = new CostBasedAutoScaler(supervisor, cfg, spec, emitter); + + // Then + Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); + } } From ac2480146d499055b0a2fb2d15d5202ccacf6004 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Wed, 31 Dec 2025 10:39:13 +0200 Subject: [PATCH 17/20] Final review comments addressed, tests optimized --- .../supervisor/SupervisorManager.java | 2 +- .../supervisor/SeekableStreamSupervisor.java | 29 +- .../autoscaler/CostBasedAutoScaler.java | 10 +- .../supervisor/SupervisorManagerTest.java | 42 +- .../autoscaler/CostBasedAutoScalerTest.java | 632 ++++++------------ .../overlord/supervisor/Supervisor.java | 4 +- 6 files changed, 243 insertions(+), 476 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index f0e959fe827d..0cd799bed547 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 @@ -502,7 +502,7 @@ private boolean createAndStartSupervisorInternal(SupervisorSpec spec, boolean pe SupervisorTaskAutoScaler autoscaler; try { supervisor = spec.createSupervisor(); - autoscaler = supervisor.createAutoscaler(); + autoscaler = supervisor.createAutoscaler(spec); supervisor.start(); if (autoscaler != 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 0bb5489548dd..0148f2225e09 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 @@ -61,6 +61,7 @@ import org.apache.druid.indexing.overlord.supervisor.StreamSupervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler; @@ -1685,7 +1686,7 @@ private List getCurrentParseErrors() } @Override - public SupervisorTaskAutoScaler createAutoscaler() + public SupervisorTaskAutoScaler createAutoscaler(SupervisorSpec spec) { this.taskAutoScaler = spec.createAutoscaler(this); return this.taskAutoScaler; @@ -3352,7 +3353,6 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException final AtomicInteger numStoppedTasks = new AtomicInteger(); // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing - // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing activelyReadingTaskGroups.entrySet().stream().sorted( Comparator.comparingLong( taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis() @@ -3391,6 +3391,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } } }); + List>> results = coalesceAndAwait(futures); for (int j = 0; j < results.size(); j++) { Integer groupId = futureGroupIds.get(j); @@ -3443,10 +3444,21 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException activelyReadingTaskGroups.remove(groupId); } + maybeScaleUpDuringTaskRollover(); + } + + /** + * Scales up or down the number of tasks during a task rollover, if applicable. + *

+ * This method is invoked to determine whether a task count adjustment is needed + * during a task rollover based on the recommendations from the task auto-scaler. + */ + private void maybeScaleUpDuringTaskRollover() + { if (taskAutoScaler != null && activelyReadingTaskGroups.isEmpty()) { int rolloverTaskCount = taskAutoScaler.computeTaskCountForRollover(); - if (rolloverTaskCount > 0 && rolloverTaskCount < ioConfig.getTaskCount()) { - log.info("Cost-based autoscaler recommends scaling down to [%d] tasks during rollover", rolloverTaskCount); + if (rolloverTaskCount > 0) { + log.info("Autoscaler recommends scaling down to [%d] tasks during rollover", rolloverTaskCount); changeTaskCountInIOConfig(rolloverTaskCount); // Here force reset the supervisor state to be re-calculated on the next iteration of runInternal() call. // This seems the best way to inject task amount recalculation during the rollover. @@ -4355,15 +4367,6 @@ public SeekableStreamSupervisorIOConfig getIoConfig() return ioConfig; } - /** - * Sets the autoscaler reference for rollover-based scale-down decisions. - * Called by {@link SupervisorManager} after supervisor creation. - */ - public void setTaskAutoScaler(@Nullable SupervisorTaskAutoScaler taskAutoScaler) - { - this.taskAutoScaler = taskAutoScaler; - } - @Override public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java index 421bd4558f63..14e3ca2cf5e8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java @@ -69,6 +69,7 @@ public class CostBasedAutoScaler implements SupervisorTaskAutoScaler private final ServiceMetricEvent.Builder metricBuilder; private final ScheduledExecutorService autoscalerExecutor; private final WeightedCostFunction costFunction; + private volatile CostMetrics lastKnownMetrics; public CostBasedAutoScaler( SeekableStreamSupervisor supervisor, @@ -128,14 +129,14 @@ public void reset() @Override public int computeTaskCountForRollover() { - return computeOptimalTaskCount(collectMetrics()); + return computeOptimalTaskCount(lastKnownMetrics); } public int computeTaskCountForScaleAction() { - CostMetrics costMetrics = collectMetrics(); - final int optimalTaskCount = computeOptimalTaskCount(costMetrics); - final int currentTaskCount = costMetrics.getCurrentTaskCount(); + lastKnownMetrics = collectMetrics(); + final int optimalTaskCount = computeOptimalTaskCount(lastKnownMetrics); + final int currentTaskCount = lastKnownMetrics.getCurrentTaskCount(); // Perform only scale-up actions return optimalTaskCount >= currentTaskCount ? optimalTaskCount : -1; @@ -213,7 +214,6 @@ int computeOptimalTaskCount(CostMetrics metrics) metrics.getPollIdleRatio() ); - if (optimalTaskCount == currentTaskCount) { return -1; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java index df1e4d37aa89..7794a798473b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManagerTest.java @@ -102,9 +102,9 @@ public void testCreateUpdateAndRemoveSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); metadataSupervisorManager.insert("id1", spec); supervisor3.start(); - EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor3.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -117,7 +117,7 @@ public void testCreateUpdateAndRemoveSupervisor() resetAll(); supervisor2.start(); - EasyMock.expect(supervisor2.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor2.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.stop(true); replayAll(); @@ -167,7 +167,7 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(ImmutableMap.of()); metadataSupervisorManager.insert("id1", spec); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -229,7 +229,7 @@ public void testShouldUpdateSupervisor() ); EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); replayAll(); manager.start(); Assert.assertFalse(manager.shouldUpdateSupervisor(spec)); @@ -254,7 +254,7 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept ); EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); exception.expect(DruidException.class); replayAll(); manager.start(); @@ -338,7 +338,7 @@ public void testGetSupervisorStatus() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.expect(supervisor1.getStatus()).andReturn(report); replayAll(); @@ -359,7 +359,7 @@ public void testHandoffTaskGroupsEarly() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.handoffTaskGroupsEarly(ImmutableList.of(1)); replayAll(); @@ -381,7 +381,7 @@ public void testHandoffTaskGroupsEarlyOnNonStreamSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor3.start(); - EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor3.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); replayAll(); @@ -423,8 +423,8 @@ public void testStartIndividualSupervisorsFailStart() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor3.start(); - EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor3.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.start(); EasyMock.expectLastCall().andThrow(new RuntimeException("supervisor explosion")); replayAll(); @@ -444,9 +444,9 @@ public void testNoPersistOnFailedStart() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(Collections.emptyMap()); metadataSupervisorManager.insert(EasyMock.eq("id1"), EasyMock.capture(capturedInsert)); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.stop(true); - EasyMock.expect(supervisor2.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor2.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor2.start(); EasyMock.expectLastCall().andThrow(new RuntimeException("supervisor failed to start")); replayAll(); @@ -474,7 +474,7 @@ public void testStopThrowsException() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.stopAsync(); EasyMock.expectLastCall().andThrow(new RuntimeException("RTE")); replayAll(); @@ -493,7 +493,7 @@ public void testResetSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.reset(EasyMock.anyObject(DataSourceMetadata.class)); replayAll(); @@ -513,7 +513,7 @@ public void testResetOnNonStreamSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor3.start(); - EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor3.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -549,7 +549,7 @@ public void testResetSupervisorWithSpecificOffsets() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.resetOffsets(datasourceMetadata); replayAll(); @@ -575,9 +575,9 @@ public void testCreateSuspendResumeAndStopSupervisor() EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs); metadataSupervisorManager.insert("id1", spec); supervisor3.start(); - EasyMock.expect(supervisor3.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor3.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); replayAll(); manager.start(); @@ -593,7 +593,7 @@ public void testCreateSuspendResumeAndStopSupervisor() resetAll(); metadataSupervisorManager.insert(EasyMock.eq("id1"), EasyMock.capture(capturedInsert)); supervisor2.start(); - EasyMock.expect(supervisor2.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor2.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); supervisor1.stop(true); replayAll(); @@ -609,7 +609,7 @@ public void testCreateSuspendResumeAndStopSupervisor() metadataSupervisorManager.insert(EasyMock.eq("id1"), EasyMock.capture(capturedInsert)); supervisor2.stop(true); supervisor1.start(); - EasyMock.expect(supervisor1.createAutoscaler()).andReturn(null).anyTimes(); + EasyMock.expect(supervisor1.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); replayAll(); manager.suspendOrResumeSupervisor("id1", false); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java index 0fc596e1bff4..6dd68f70dd4f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java @@ -71,12 +71,23 @@ public void testComputeValidTaskCounts() { // For 100 partitions at 25 tasks (4 partitions/task), valid counts include 25 and 34 int[] validTaskCounts = CostBasedAutoScaler.computeValidTaskCounts(100, 25); - Assert.assertTrue("Should contain the current task count", contains(validTaskCounts, 25)); Assert.assertTrue("Should contain the next scale-up option", contains(validTaskCounts, 34)); - // Edge case: zero partitions return an empty array - Assert.assertEquals(0, CostBasedAutoScaler.computeValidTaskCounts(0, 10).length); + // Edge cases + Assert.assertEquals("Zero partitions return empty array", 0, CostBasedAutoScaler.computeValidTaskCounts(0, 10).length); + Assert.assertEquals("Negative partitions return empty array", 0, CostBasedAutoScaler.computeValidTaskCounts(-5, 10).length); + + // Single partition + int[] singlePartition = CostBasedAutoScaler.computeValidTaskCounts(1, 1); + Assert.assertTrue("Single partition should have at least one valid count", singlePartition.length > 0); + Assert.assertTrue("Single partition should contain 1", contains(singlePartition, 1)); + + // Current exceeds partitions - should still yield valid, deduplicated options + int[] exceedsPartitions = CostBasedAutoScaler.computeValidTaskCounts(2, 5); + Assert.assertEquals(2, exceedsPartitions.length); + Assert.assertTrue(contains(exceedsPartitions, 1)); + Assert.assertTrue(contains(exceedsPartitions, 2)); } @Test @@ -84,28 +95,24 @@ public void testComputeOptimalTaskCountInvalidInputs() { Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(null)); Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(0.0, 10, 0, 0.0))); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(100.0, 10, -5, 0.3))); + Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(createMetrics(100.0, -1, 100, 0.3))); } @Test public void testComputeOptimalTaskCountScaling() { // High idle (underutilized) - should scale down - // With high idle (0.8), the algorithm evaluates lower task counts and finds they have lower idle cost int scaleDownResult = autoScaler.computeOptimalTaskCount(createMetrics(100.0, 25, 100, 0.8)); Assert.assertTrue("Should scale down when idle > 0.6", scaleDownResult < 25); - } - @Test - public void testComputeOptimalTaskCountLowIdleDoesNotScaleUpWithBalancedWeights() - { - // With a corrected idle ratio model and marginal lag model, low idle does not automatically trigger scale-up. - int result = autoScaler.computeOptimalTaskCount(createMetrics(1000.0, 25, 100, 0.1)); + // Very high idle with high task count - should scale down + int highIdleResult = autoScaler.computeOptimalTaskCount(createMetrics(10.0, 50, 100, 0.9)); + Assert.assertTrue("Scale down scenario should return optimal <= current", highIdleResult <= 50); - // Algorithm evaluates costs and may find the current count optimal - // or may scale down if idle cost reduction outweighs lag increase. - Assert.assertTrue( - "With low idle and balanced weights, algorithm should not scale up aggressively", result <= 25 - ); + // With low idle and balanced weights, algorithm should not scale up aggressively + int lowIdleResult = autoScaler.computeOptimalTaskCount(createMetrics(1000.0, 25, 100, 0.1)); + Assert.assertTrue("With low idle and balanced weights, should not scale up aggressively", lowIdleResult <= 25); } @Test @@ -130,26 +137,50 @@ public void testExtractPollIdleRatio() } @Test - public void testExtractProcessingRateMovingAverage() + public void testExtractPollIdleRatioInvalidTypes() + { + // Non-map task metric + Map> nonMapTask = new HashMap<>(); + nonMapTask.put("0", Collections.singletonMap("task-0", "not-a-map")); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(nonMapTask), 0.0001); + + // Empty autoscaler metrics + Map> emptyAutoscaler = new HashMap<>(); + Map taskStats1 = new HashMap<>(); + taskStats1.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, new HashMap<>()); + emptyAutoscaler.put("0", Collections.singletonMap("task-0", taskStats1)); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(emptyAutoscaler), 0.0001); + + // Non-map autoscaler metrics + Map> nonMapAutoscaler = new HashMap<>(); + Map taskStats2 = new HashMap<>(); + taskStats2.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, "not-a-map"); + nonMapAutoscaler.put("0", Collections.singletonMap("task-0", taskStats2)); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(nonMapAutoscaler), 0.0001); + + // Non-number poll idle ratio + Map> nonNumberRatio = new HashMap<>(); + Map taskStats3 = new HashMap<>(); + Map autoscalerMetrics = new HashMap<>(); + autoscalerMetrics.put(SeekableStreamIndexTaskRunner.POLL_IDLE_RATIO_KEY, "not-a-number"); + taskStats3.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, autoscalerMetrics); + nonNumberRatio.put("0", Collections.singletonMap("task-0", taskStats3)); + Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(nonNumberRatio), 0.0001); + } + + @Test + public void testExtractMovingAverage() { // Null and empty return -1 - Assert.assertEquals( - -1., - CostBasedAutoScaler.extractMovingAverage(null), - 0.0001 - ); - Assert.assertEquals( - -1., - CostBasedAutoScaler.extractMovingAverage(Collections.emptyMap()), - 0.0001 - ); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(null), 0.0001); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(Collections.emptyMap()), 0.0001); // Missing metrics return -1 Map> missingMetrics = new HashMap<>(); missingMetrics.put("0", Collections.singletonMap("task-0", new HashMap<>())); Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(missingMetrics), 0.0001); - // Valid stats return average + // Valid stats return average (using 5-minute) Map> validStats = new HashMap<>(); Map group = new HashMap<>(); group.put("task-0", buildTaskStatsWithMovingAverage(1000.0)); @@ -159,361 +190,139 @@ public void testExtractProcessingRateMovingAverage() } @Test - public void testExtractMovingAverageFifteenMinuteFallback() - { - // Test that 15-minute average is preferred when available - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - group.put("task-0", buildTaskStatsWithMovingAverageForInterval(FIFTEEN_MINUTE_NAME, 1500.0)); - stats.put("0", group); - Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } - - @Test - public void testExtractMovingAverageOneMinuteFallback() - { - // Test that 1-minute average is used as final fallback - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - group.put( - "task-0", - buildTaskStatsWithMovingAverageForInterval(ONE_MINUTE_NAME, 500.0) - ); - stats.put("0", group); - Assert.assertEquals(500.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } - - @Test - public void testExtractMovingAveragePrefersFifteenOverFive() - { - // Test that 15-minute average is preferred over 5-minute when both are available - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - group.put("task-0", buildTaskStatsWithMultipleMovingAverages(1500.0, 1000.0, 500.0)); - stats.put("0", group); - // Should use 15-minute average (1500.0), not 5-minute (1000.0) or 1-minute (500.0) - Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } - - @Test - public void testComputeTaskCountForScaleActionScaleUp() - { - // Test scale-up scenario: optimal > current - // With low idle ratio and high lag, should want to scale up - CostMetrics highLagMetrics = new CostMetrics( - 10000.0, // high lag - 5, // current task count - 100, // partition count - 0.1, // low idle ratio (busy) - 3600, - 1000.0 - ); - - int result = autoScaler.computeOptimalTaskCount(highLagMetrics); - // The algorithm should evaluate different task counts - Assert.assertTrue("Should return a valid task count", result >= -1); - } - - @Test - public void testComputeTaskCountForScaleActionNoScale() - { - // Test no-scale scenario: optimal == current - CostMetrics balancedMetrics = new CostMetrics( - 100.0, // moderate lag - 25, // current task count - 100, // partition count - 0.4, // moderate idle ratio - 3600, - 1000.0 - ); - - int result = autoScaler.computeOptimalTaskCount(balancedMetrics); - // Either returns -1 (no change) or a different task count - Assert.assertTrue("Result should be -1 or a valid positive number", result >= -1); - } - - @Test - public void testComputeOptimalTaskCountWithNegativePartitions() - { - CostMetrics invalidMetrics = new CostMetrics( - 100.0, - 10, - -5, // negative partition count - 0.3, - 3600, - 1000.0 - ); - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(invalidMetrics)); - } - - @Test - public void testComputeOptimalTaskCountWithNegativeTaskCount() - { - CostMetrics invalidMetrics = new CostMetrics( - 100.0, - -1, // negative task count - 100, - 0.3, - 3600, - 1000.0 - ); - Assert.assertEquals(-1, autoScaler.computeOptimalTaskCount(invalidMetrics)); - } - - @Test - public void testComputeValidTaskCountsWithSinglePartition() - { - // Edge case: single partition - int[] validTaskCounts = CostBasedAutoScaler.computeValidTaskCounts(1, 1); - Assert.assertTrue("Should have at least one valid count", validTaskCounts.length > 0); - Assert.assertTrue("Should contain 1 as valid count", contains(validTaskCounts, 1)); - } - - @Test - public void testComputeValidTaskCountsWithNegativePartitions() - { - // Negative partitions should return empty array - int[] validTaskCounts = CostBasedAutoScaler.computeValidTaskCounts(-5, 10); - Assert.assertEquals(0, validTaskCounts.length); - } - - @Test - public void testExtractPollIdleRatioWithNonMapTaskMetric() - { - // Test branch where taskMetric is not a Map - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - group.put("task-0", "not-a-map"); - stats.put("0", group); - Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); - } - - @Test - public void testExtractPollIdleRatioWithMissingAutoscalerMetrics() - { - // Test branch where autoscaler metrics map is present but poll idle ratio is missing - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map emptyAutoscalerMetrics = new HashMap<>(); - taskStats.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, emptyAutoscalerMetrics); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); - } - - @Test - public void testExtractMovingAverageWithNonMapTaskMetric() - { - // Test branch where taskMetric is not a Map - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - group.put("task-0", "not-a-map"); - stats.put("0", group); - Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } - - @Test - public void testExtractMovingAverageWithMissingBuildSegments() - { - // Test branch where movingAverages exists but buildSegments is missing - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map movingAverages = new HashMap<>(); - taskStats.put("movingAverages", movingAverages); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } - - @Test - public void testExtractMovingAverageWithNonMapMovingAverage() - { - // Test branch where movingAveragesObj is not a Map - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - taskStats.put("movingAverages", "not-a-map"); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } - - @Test - public void testComputeTaskCountForScaleActionReturnsMinusOneWhenScaleDown() - { - // When optimal < current, computeTaskCountForScaleAction should return -1 - // This tests the ternary: optimalTaskCount >= currentTaskCount ? optimalTaskCount : -1 - // Create a scenario where the algorithm wants to scale down (high idle ratio) - CostMetrics highIdleMetrics = new CostMetrics( - 10.0, // low lag - 50, // current task count (high) - 100, // partition count - 0.9, // very high idle ratio (underutilized) - 3600, - 1000.0 - ); - - // computeOptimalTaskCount may return a lower task count - int optimalResult = autoScaler.computeOptimalTaskCount(highIdleMetrics); - // The test verifies that computeTaskCountForScaleAction handles scale-down correctly - Assert.assertTrue("Scale down scenario should return optimal <= current", optimalResult <= 50); - } - - @Test - public void testComputeTaskCountForScaleActionReturnsPositiveWhenScaleUp() - { - // When optimal > current, computeTaskCountForScaleAction should return the optimal value - // Create a scenario with low idle (tasks are busy) and some lag - CostMetrics busyMetrics = new CostMetrics( - 5000.0, // significant lag - 5, // low current task count - 100, // partition count (20 partitions per task) - 0.05, // very low idle ratio (tasks are very busy) - 3600, - 1000.0 - ); - - int optimalResult = autoScaler.computeOptimalTaskCount(busyMetrics); - // With very low idle ratio, algorithm should evaluate higher task counts - Assert.assertTrue("Busy scenario result should be valid", optimalResult >= -1); + public void testExtractMovingAverageIntervalFallback() + { + // 15-minute average is preferred + Map> fifteenMin = new HashMap<>(); + fifteenMin.put("0", Collections.singletonMap("task-0", buildTaskStatsWithMovingAverageForInterval(FIFTEEN_MINUTE_NAME, 1500.0))); + Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(fifteenMin), 0.0001); + + // 1-minute as final fallback + Map> oneMin = new HashMap<>(); + oneMin.put("0", Collections.singletonMap("task-0", buildTaskStatsWithMovingAverageForInterval(ONE_MINUTE_NAME, 500.0))); + Assert.assertEquals(500.0, CostBasedAutoScaler.extractMovingAverage(oneMin), 0.0001); + + // 15-minute preferred over 5-minute when both available + Map> allIntervals = new HashMap<>(); + allIntervals.put("0", Collections.singletonMap("task-0", buildTaskStatsWithMultipleMovingAverages(1500.0, 1000.0, 500.0))); + Assert.assertEquals(1500.0, CostBasedAutoScaler.extractMovingAverage(allIntervals), 0.0001); + + // Falls back to 5-minute when 15-minute is null + Map> nullFifteen = new HashMap<>(); + nullFifteen.put("0", Collections.singletonMap("task-0", buildTaskStatsWithNullInterval(FIFTEEN_MINUTE_NAME, FIVE_MINUTE_NAME, 750.0))); + Assert.assertEquals(750.0, CostBasedAutoScaler.extractMovingAverage(nullFifteen), 0.0001); + + // Falls back to 1-minute when both 15 and 5 are null + Map> bothNull = new HashMap<>(); + bothNull.put("0", Collections.singletonMap("task-0", buildTaskStatsWithTwoNullIntervals(250.0))); + Assert.assertEquals(250.0, CostBasedAutoScaler.extractMovingAverage(bothNull), 0.0001); + } + + @Test + public void testExtractMovingAverageInvalidTypes() + { + // Non-map task metric + Map> nonMapTask = new HashMap<>(); + nonMapTask.put("0", Collections.singletonMap("task-0", "not-a-map")); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(nonMapTask), 0.0001); + + // Missing buildSegments + Map> missingBuild = new HashMap<>(); + Map taskStats1 = new HashMap<>(); + taskStats1.put("movingAverages", new HashMap<>()); + missingBuild.put("0", Collections.singletonMap("task-0", taskStats1)); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(missingBuild), 0.0001); + + // Non-map movingAverages + Map> nonMapMA = new HashMap<>(); + Map taskStats2 = new HashMap<>(); + taskStats2.put("movingAverages", "not-a-map"); + nonMapMA.put("0", Collections.singletonMap("task-0", taskStats2)); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(nonMapMA), 0.0001); + + // Non-map buildSegments + Map> nonMapBS = new HashMap<>(); + Map taskStats3 = new HashMap<>(); + Map movingAverages3 = new HashMap<>(); + movingAverages3.put(RowIngestionMeters.BUILD_SEGMENTS, "not-a-map"); + taskStats3.put("movingAverages", movingAverages3); + nonMapBS.put("0", Collections.singletonMap("task-0", taskStats3)); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(nonMapBS), 0.0001); + + // Non-map interval data + Map> nonMapInterval = new HashMap<>(); + Map taskStats4 = new HashMap<>(); + Map movingAverages4 = new HashMap<>(); + Map buildSegments4 = new HashMap<>(); + buildSegments4.put(FIFTEEN_MINUTE_NAME, "not-a-map"); + movingAverages4.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments4); + taskStats4.put("movingAverages", movingAverages4); + nonMapInterval.put("0", Collections.singletonMap("task-0", taskStats4)); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(nonMapInterval), 0.0001); + + // Non-number processed rate + Map> nonNumberRate = new HashMap<>(); + Map taskStats5 = new HashMap<>(); + Map movingAverages5 = new HashMap<>(); + Map buildSegments5 = new HashMap<>(); + Map fifteenMin = new HashMap<>(); + fifteenMin.put(RowIngestionMeters.PROCESSED, "not-a-number"); + buildSegments5.put(FIFTEEN_MINUTE_NAME, fifteenMin); + movingAverages5.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments5); + taskStats5.put("movingAverages", movingAverages5); + nonNumberRate.put("0", Collections.singletonMap("task-0", taskStats5)); + Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(nonNumberRate), 0.0001); } @Test - public void testComputeOptimalTaskCountWhenOptimalEqualsCurrent() + public void testComputeTaskCountForRolloverReturnsMinusOneWhenSuspended() { - // Test the branch where optimalTaskCount == currentTaskCount returns -1 - // Create balanced metrics that likely result in current count being optimal - CostMetrics balancedMetrics = new CostMetrics( - 50.0, // low lag - 20, // current task count - 100, // partition count (5 partitions per task) - 0.5, // moderate idle ratio - 3600, - 1000.0 - ); - - int result = autoScaler.computeOptimalTaskCount(balancedMetrics); - // Either -1 (optimal == current) or a different task count - Assert.assertTrue("Result should be -1 or positive", result >= -1); - } + SupervisorSpec spec = Mockito.mock(SupervisorSpec.class); + SeekableStreamSupervisor supervisor = Mockito.mock(SeekableStreamSupervisor.class); + ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + SeekableStreamSupervisorIOConfig ioConfig = Mockito.mock(SeekableStreamSupervisorIOConfig.class); - @Test - public void testExtractPollIdleRatioWithNonMapAutoscalerMetrics() - { - // Test branch where AUTOSCALER_METRICS_KEY exists but is not a Map - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - taskStats.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, "not-a-map"); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); - } + when(spec.getId()).thenReturn("s-up"); + when(spec.isSuspended()).thenReturn(true); + when(supervisor.getIoConfig()).thenReturn(ioConfig); + when(ioConfig.getStream()).thenReturn("stream"); - @Test - public void testExtractPollIdleRatioWithNonNumberPollIdleRatio() - { - // Test branch where pollIdleRatioAvg exists but is not a Number - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map autoscalerMetrics = new HashMap<>(); - autoscalerMetrics.put(SeekableStreamIndexTaskRunner.POLL_IDLE_RATIO_KEY, "not-a-number"); - taskStats.put(SeekableStreamIndexTaskRunner.AUTOSCALER_METRICS_KEY, autoscalerMetrics); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(0., CostBasedAutoScaler.extractPollIdleRatio(stats), 0.0001); - } + CostBasedAutoScalerConfig cfg = CostBasedAutoScalerConfig.builder() + .taskCountMax(10) + .taskCountMin(1) + .enableTaskAutoScaler(true) + .lagWeight(0.5) + .idleWeight(0.5) + .build(); - @Test - public void testExtractMovingAverageWithNonMapBuildSegments() - { - // Test branch where buildSegmentsObj is not a Map - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map movingAverages = new HashMap<>(); - movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, "not-a-map"); - taskStats.put("movingAverages", movingAverages); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + CostBasedAutoScaler scaler = new CostBasedAutoScaler(supervisor, cfg, spec, emitter); + Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); } @Test - public void testExtractMovingAverageWithNonMapIntervalData() + public void testComputeTaskCountForRolloverReturnsMinusOneWhenLagStatsNull() { - // Test branch where the 15min/5min/1min interval data is not a Map - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map movingAverages = new HashMap<>(); - Map buildSegments = new HashMap<>(); - buildSegments.put(FIFTEEN_MINUTE_NAME, "not-a-map"); - movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); - taskStats.put("movingAverages", movingAverages); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } + SupervisorSpec spec = Mockito.mock(SupervisorSpec.class); + SeekableStreamSupervisor supervisor = Mockito.mock(SeekableStreamSupervisor.class); + ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + SeekableStreamSupervisorIOConfig ioConfig = Mockito.mock(SeekableStreamSupervisorIOConfig.class); - @Test - public void testExtractMovingAverageWithNonNumberProcessedRate() - { - // Test branch where processedRate is not a Number - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map movingAverages = new HashMap<>(); - Map buildSegments = new HashMap<>(); - Map fifteenMin = new HashMap<>(); - fifteenMin.put(RowIngestionMeters.PROCESSED, "not-a-number"); - buildSegments.put(FIFTEEN_MINUTE_NAME, fifteenMin); - movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); - taskStats.put("movingAverages", movingAverages); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(-1., CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } + when(spec.getId()).thenReturn("s-up"); + when(spec.isSuspended()).thenReturn(false); + when(supervisor.computeLagStats()).thenReturn(null); + when(supervisor.getIoConfig()).thenReturn(ioConfig); + when(ioConfig.getStream()).thenReturn("stream"); - @Test - public void testExtractMovingAverageFallsBackToFiveMinuteWhenFifteenMinuteNull() - { - // Test the fallback from 15min to 5min when 15min is explicitly null - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map movingAverages = new HashMap<>(); - Map buildSegments = new HashMap<>(); - // Explicitly set 15min to null (not just missing) - buildSegments.put(FIFTEEN_MINUTE_NAME, null); - buildSegments.put(FIVE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, 750.0)); - movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); - taskStats.put("movingAverages", movingAverages); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(750.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); - } + CostBasedAutoScalerConfig cfg = CostBasedAutoScalerConfig.builder() + .taskCountMax(10) + .taskCountMin(1) + .enableTaskAutoScaler(true) + .lagWeight(0.5) + .idleWeight(0.5) + .build(); - @Test - public void testExtractMovingAverageFallsBackToOneMinuteWhenBothNull() - { - // Test the fallback from 15min to 5min to 1min when both 15min and 5min are null - Map> stats = new HashMap<>(); - Map group = new HashMap<>(); - Map taskStats = new HashMap<>(); - Map movingAverages = new HashMap<>(); - Map buildSegments = new HashMap<>(); - buildSegments.put(FIFTEEN_MINUTE_NAME, null); - buildSegments.put(FIVE_MINUTE_NAME, null); - buildSegments.put(ONE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, 250.0)); - movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); - taskStats.put("movingAverages", movingAverages); - group.put("task-0", taskStats); - stats.put("0", group); - Assert.assertEquals(250.0, CostBasedAutoScaler.extractMovingAverage(stats), 0.0001); + CostBasedAutoScaler scaler = new CostBasedAutoScaler(supervisor, cfg, spec, emitter); + Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); } private CostMetrics createMetrics( @@ -556,10 +365,7 @@ private Map buildTaskStatsWithPollIdle(double pollIdleRatio) private Map buildTaskStatsWithMovingAverage(double processedRate) { Map buildSegments = new HashMap<>(); - buildSegments.put( - FIVE_MINUTE_NAME, - Map.of(RowIngestionMeters.PROCESSED, processedRate) - ); + buildSegments.put(FIVE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, processedRate)); Map movingAverages = new HashMap<>(); movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); @@ -589,10 +395,7 @@ private Map buildTaskStatsWithMultipleMovingAverages( ) { Map buildSegments = new HashMap<>(); - buildSegments.put( - FIFTEEN_MINUTE_NAME, - Map.of(RowIngestionMeters.PROCESSED, fifteenMinRate) - ); + buildSegments.put(FIFTEEN_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, fifteenMinRate)); buildSegments.put(FIVE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, fiveMinRate)); buildSegments.put(ONE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, oneMinRate)); @@ -604,71 +407,32 @@ private Map buildTaskStatsWithMultipleMovingAverages( return taskStats; } - @Test - public void testComputeValidTaskCountsWhenCurrentExceedsPartitions() - { - // the currentTaskCount > partitionCount should still yield valid, - // deduplicated options - int[] counts = CostBasedAutoScaler.computeValidTaskCounts(2, 5); - Assert.assertEquals(2, counts.length); - Assert.assertTrue(contains(counts, 1)); - Assert.assertTrue(contains(counts, 2)); - } - - @Test - public void testComputeTaskCountForRolloverReturnsMinusOneWhenSuspended() + private Map buildTaskStatsWithNullInterval(String nullInterval, String validInterval, double processedRate) { - // Arrange: build autoscaler with suspended spec so collectMetrics returns null - SupervisorSpec spec = Mockito.mock(SupervisorSpec.class); - SeekableStreamSupervisor supervisor = Mockito.mock(SeekableStreamSupervisor.class); - ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); - SeekableStreamSupervisorIOConfig ioConfig = Mockito.mock(SeekableStreamSupervisorIOConfig.class); - - when(spec.getId()).thenReturn("s-up"); - when(spec.isSuspended()).thenReturn(true); - when(supervisor.getIoConfig()).thenReturn(ioConfig); - when(ioConfig.getStream()).thenReturn("stream"); - - CostBasedAutoScalerConfig cfg = CostBasedAutoScalerConfig.builder() - .taskCountMax(10) - .taskCountMin(1) - .enableTaskAutoScaler(true) - .lagWeight(0.5) - .idleWeight(0.5) - .build(); + Map buildSegments = new HashMap<>(); + buildSegments.put(nullInterval, null); + buildSegments.put(validInterval, Map.of(RowIngestionMeters.PROCESSED, processedRate)); - CostBasedAutoScaler scaler = new CostBasedAutoScaler(supervisor, cfg, spec, emitter); + Map movingAverages = new HashMap<>(); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); - // Then - Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); + Map taskStats = new HashMap<>(); + taskStats.put("movingAverages", movingAverages); + return taskStats; } - @Test - public void testComputeTaskCountForRolloverReturnsMinusOneWhenLagStatsNull() + private Map buildTaskStatsWithTwoNullIntervals(double oneMinRate) { - // Arrange: collectMetrics should early-return when lagStats is null - SupervisorSpec spec = Mockito.mock(SupervisorSpec.class); - SeekableStreamSupervisor supervisor = Mockito.mock(SeekableStreamSupervisor.class); - ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); - SeekableStreamSupervisorIOConfig ioConfig = Mockito.mock(SeekableStreamSupervisorIOConfig.class); - - when(spec.getId()).thenReturn("s-up"); - when(spec.isSuspended()).thenReturn(false); - when(supervisor.computeLagStats()).thenReturn(null); - when(supervisor.getIoConfig()).thenReturn(ioConfig); - when(ioConfig.getStream()).thenReturn("stream"); - - CostBasedAutoScalerConfig cfg = CostBasedAutoScalerConfig.builder() - .taskCountMax(10) - .taskCountMin(1) - .enableTaskAutoScaler(true) - .lagWeight(0.5) - .idleWeight(0.5) - .build(); + Map buildSegments = new HashMap<>(); + buildSegments.put(FIFTEEN_MINUTE_NAME, null); + buildSegments.put(FIVE_MINUTE_NAME, null); + buildSegments.put(ONE_MINUTE_NAME, Map.of(RowIngestionMeters.PROCESSED, oneMinRate)); - CostBasedAutoScaler scaler = new CostBasedAutoScaler(supervisor, cfg, spec, emitter); + Map movingAverages = new HashMap<>(); + movingAverages.put(RowIngestionMeters.BUILD_SEGMENTS, buildSegments); - // Then - Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); + Map taskStats = new HashMap<>(); + taskStats.put("movingAverages", movingAverages); + return taskStats; } } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java index 44dcdc9d7147..ce6c87e5e082 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java @@ -87,9 +87,9 @@ default Boolean isHealthy() return null; // default implementation for interface compatability; returning null since true or false is misleading } - default SupervisorTaskAutoScaler createAutoscaler() + default SupervisorTaskAutoScaler createAutoscaler(SupervisorSpec spec) { - return null; + return spec.createAutoscaler(this); } /** From 2453e66efb049915d02fdbb9c2296b63fbabf3d4 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Wed, 31 Dec 2025 15:52:11 +0200 Subject: [PATCH 18/20] Increase branch coverage --- .../SeekableStreamSupervisorStateTest.java | 22 +++++++++++++++ .../autoscaler/CostBasedAutoScalerTest.java | 27 +++++++++++++++++++ 2 files changed, 49 insertions(+) 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 5df9edd184d2..71e798e7e2fa 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 @@ -2751,6 +2751,28 @@ public void testMaxAllowedStopsWithStopTaskCountRatio() Assert.assertEquals(12, config.getMaxAllowedStops()); } + @Test + public void testCreateAutoscalerStoresAndReturnsAutoscaler() + { + EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); + + replayAll(); + + SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); + + // Test that createAutoscaler returns null when spec returns null + SeekableStreamSupervisorSpec mockSpec = EasyMock.createMock(SeekableStreamSupervisorSpec.class); + EasyMock.expect(mockSpec.createAutoscaler(supervisor)).andReturn(null).once(); + EasyMock.replay(mockSpec); + + Assert.assertNull(supervisor.createAutoscaler(mockSpec)); + EasyMock.verify(mockSpec); + + verifyAll(); + } + private static DataSchema getDataSchema() { List dimensions = new ArrayList<>(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java index 6dd68f70dd4f..caf5453f5217 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerTest.java @@ -325,6 +325,33 @@ public void testComputeTaskCountForRolloverReturnsMinusOneWhenLagStatsNull() Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); } + @Test + public void testComputeTaskCountForRolloverReturnsMinusOneWhenNoMetrics() + { + // Tests the case where lastKnownMetrics is null (no computeTaskCountForScaleAction called) + SupervisorSpec spec = Mockito.mock(SupervisorSpec.class); + SeekableStreamSupervisor supervisor = Mockito.mock(SeekableStreamSupervisor.class); + ServiceEmitter emitter = Mockito.mock(ServiceEmitter.class); + SeekableStreamSupervisorIOConfig ioConfig = Mockito.mock(SeekableStreamSupervisorIOConfig.class); + + when(spec.getId()).thenReturn("s-up"); + when(spec.isSuspended()).thenReturn(false); + when(supervisor.getIoConfig()).thenReturn(ioConfig); + when(ioConfig.getStream()).thenReturn("stream"); + + CostBasedAutoScalerConfig cfg = CostBasedAutoScalerConfig.builder() + .taskCountMax(10) + .taskCountMin(1) + .enableTaskAutoScaler(true) + .lagWeight(0.5) + .idleWeight(0.5) + .build(); + + CostBasedAutoScaler scaler = new CostBasedAutoScaler(supervisor, cfg, spec, emitter); + // Should return -1 when lastKnownMetrics is null + Assert.assertEquals(-1, scaler.computeTaskCountForRollover()); + } + private CostMetrics createMetrics( double avgPartitionLag, int currentTaskCount, From 851807e7179fd1ad0d81450744b2f11920a0cd9f Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Mon, 5 Jan 2026 11:22:01 +0200 Subject: [PATCH 19/20] A bit of test restructuring --- .../SeekableStreamSupervisorSpecTest.java | 338 +++++++++--------- 1 file changed, 169 insertions(+), 169 deletions(-) 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 f5b925e2c59d..f0b15f8c8a09 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 @@ -956,71 +956,6 @@ public int getActiveTaskGroupsCount() autoScaler.stop(); } - @Test - public void test_dynamicAllocationNotice_skipsScalingAndEmitsReason_ifTasksArePublishing() throws InterruptedException - { - EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); - EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); - EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); - EasyMock.expect(spec.getIoConfig()).andReturn(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.replay(spec); - - EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); - EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); - EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); - EasyMock.replay(ingestionSchema); - - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); - EasyMock.replay(taskMaster); - - StubServiceEmitter dynamicActionEmitter = new StubServiceEmitter(); - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(10); - - LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( - supervisor, - DATASOURCE, - mapper.convertValue( - getScaleOutProperties(2), - LagBasedAutoScalerConfig.class - ), - spec, - dynamicActionEmitter - ); - - supervisor.addTaskGroupToPendingCompletionTaskGroup( - 0, - ImmutableMap.of("0", "0"), - null, - null, - Set.of("dummyTask"), - Collections.emptySet() - ); - - supervisor.start(); - autoScaler.start(); - - supervisor.runInternal(); - Thread.sleep(1000); // ensure a dynamic allocation notice completes - - Assert.assertEquals(1, supervisor.getIoConfig().getTaskCount().intValue()); - Assert.assertTrue( - dynamicActionEmitter - .getMetricEvents(SeekableStreamSupervisor.AUTOSCALER_REQUIRED_TASKS_METRIC) - .stream() - .map(metric -> metric.getUserDims().get(SeekableStreamSupervisor.AUTOSCALER_SKIP_REASON_DIMENSION)) - .filter(Objects::nonNull) - .anyMatch("There are tasks pending completion"::equals) - ); - - emitter.verifyNotEmitted(SeekableStreamSupervisor.AUTOSCALER_SCALING_TIME_METRIC); - autoScaler.reset(); - autoScaler.stop(); - } - @Test public void testSeekableStreamSupervisorSpecWithNoScalingOnIdleSupervisor() throws InterruptedException { @@ -1071,110 +1006,6 @@ public void testSeekableStreamSupervisorSpecWithNoScalingOnIdleSupervisor() thro autoScaler.stop(); } - @Test - public void test_dynamicAllocationNotice_skips_whenSupervisorSuspended() throws InterruptedException - { - EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); - EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); - - EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); - EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig(1, true)).anyTimes(); - EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); - EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); - // Suspended → DynamicAllocationTasksNotice should return early and not scale - EasyMock.expect(spec.isSuspended()).andReturn(true).anyTimes(); - EasyMock.replay(spec); - - EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); - EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); - EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); - EasyMock.replay(ingestionSchema); - - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); - EasyMock.replay(taskMaster); - - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); - LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( - supervisor, - DATASOURCE, - mapper.convertValue( - getScaleOutProperties(2), - LagBasedAutoScalerConfig.class - ), - spec, - emitter - ); - - supervisor.start(); - autoScaler.start(); - supervisor.runInternal(); - - int before = supervisor.getIoConfig().getTaskCount(); - Thread.sleep(1000); - int after = supervisor.getIoConfig().getTaskCount(); - // No scaling expected because supervisor is suspended - Assert.assertEquals(before, after); - - autoScaler.reset(); - autoScaler.stop(); - } - - @Test - public void test_changeTaskCountInIOConfig_handlesExceptionAndStillUpdatesTaskCount() throws InterruptedException - { - EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); - EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); - - EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); - EasyMock.expect(spec.getIoConfig()).andReturn(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.replay(spec); - - EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); - EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); - EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); - EasyMock.replay(ingestionSchema); - - // SupervisorManager present but metadata insert fails → should be handled - SupervisorManager sm = EasyMock.createMock(SupervisorManager.class); - MetadataSupervisorManager msm = EasyMock.createMock(MetadataSupervisorManager.class); - EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); - EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(sm)).anyTimes(); - EasyMock.expect(sm.getMetadataSupervisorManager()).andReturn(msm).anyTimes(); - msm.insert(EasyMock.anyString(), EasyMock.anyObject()); - EasyMock.expectLastCall().andThrow(new RuntimeException("boom")).anyTimes(); - EasyMock.replay(taskMaster, sm, msm); - - TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(10); - LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( - supervisor, - DATASOURCE, - mapper.convertValue( - getScaleOutProperties(2), - LagBasedAutoScalerConfig.class - ), - spec, - emitter - ); - - supervisor.start(); - autoScaler.start(); - supervisor.runInternal(); - - int before = supervisor.getIoConfig().getTaskCount(); - Assert.assertEquals(1, before); - Thread.sleep(1000); // allow one dynamic allocation cycle - int after = supervisor.getIoConfig().getTaskCount(); - // Even though metadata insert failed, taskCount should still be updated in ioConfig - Assert.assertEquals(2, after); - - autoScaler.reset(); - autoScaler.stop(); - } - @Test public void testSeekableStreamSupervisorSpecWithScaleOutSmallPartitionNumber() throws InterruptedException { @@ -1796,6 +1627,175 @@ public String getSource() originalSpec.validateSpecUpdateTo(proposedSpecSameSource); } + @Test + public void test_dynamicAllocationNotice_skipsScalingAndEmitsReason_ifTasksArePublishing() throws InterruptedException + { + EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(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.replay(spec); + + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.replay(taskMaster); + + StubServiceEmitter dynamicActionEmitter = new StubServiceEmitter(); + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(10); + + LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( + supervisor, + DATASOURCE, + mapper.convertValue( + getScaleOutProperties(2), + LagBasedAutoScalerConfig.class + ), + spec, + dynamicActionEmitter + ); + + supervisor.addTaskGroupToPendingCompletionTaskGroup( + 0, + ImmutableMap.of("0", "0"), + null, + null, + Set.of("dummyTask"), + Collections.emptySet() + ); + + supervisor.start(); + autoScaler.start(); + + supervisor.runInternal(); + Thread.sleep(1000); // ensure a dynamic allocation notice completes + + Assert.assertEquals(1, supervisor.getIoConfig().getTaskCount().intValue()); + Assert.assertTrue( + dynamicActionEmitter + .getMetricEvents(SeekableStreamSupervisor.AUTOSCALER_REQUIRED_TASKS_METRIC) + .stream() + .map(metric -> metric.getUserDims().get(SeekableStreamSupervisor.AUTOSCALER_SKIP_REASON_DIMENSION)) + .filter(Objects::nonNull) + .anyMatch("There are tasks pending completion"::equals) + ); + + emitter.verifyNotEmitted(SeekableStreamSupervisor.AUTOSCALER_SCALING_TIME_METRIC); + autoScaler.reset(); + autoScaler.stop(); + } + + @Test + public void test_dynamicAllocationNotice_skips_whenSupervisorSuspended() throws InterruptedException + { + EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig(1, true)).anyTimes(); + EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); + EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + // Suspended → DynamicAllocationTasksNotice should return early and not scale + EasyMock.expect(spec.isSuspended()).andReturn(true).anyTimes(); + EasyMock.replay(spec); + + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.replay(taskMaster); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(3); + LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( + supervisor, + DATASOURCE, + mapper.convertValue( + getScaleOutProperties(2), + LagBasedAutoScalerConfig.class + ), + spec, + emitter + ); + + supervisor.start(); + autoScaler.start(); + supervisor.runInternal(); + + int before = supervisor.getIoConfig().getTaskCount(); + Thread.sleep(1000); + int after = supervisor.getIoConfig().getTaskCount(); + // No scaling expected because supervisor is suspended + Assert.assertEquals(before, after); + + autoScaler.reset(); + autoScaler.stop(); + } + + @Test + public void test_changeTaskCountInIOConfig_handlesExceptionAndStillUpdatesTaskCount() throws InterruptedException + { + EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(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.replay(spec); + + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(seekableStreamSupervisorTuningConfig).anyTimes(); + EasyMock.replay(ingestionSchema); + + // SupervisorManager present but metadata insert fails → should be handled + SupervisorManager sm = EasyMock.createMock(SupervisorManager.class); + MetadataSupervisorManager msm = EasyMock.createMock(MetadataSupervisorManager.class); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(sm)).anyTimes(); + EasyMock.expect(sm.getMetadataSupervisorManager()).andReturn(msm).anyTimes(); + msm.insert(EasyMock.anyString(), EasyMock.anyObject()); + EasyMock.expectLastCall().andThrow(new RuntimeException("boom")).anyTimes(); + EasyMock.replay(taskMaster, sm, msm); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(10); + LagBasedAutoScaler autoScaler = new LagBasedAutoScaler( + supervisor, + DATASOURCE, + mapper.convertValue( + getScaleOutProperties(2), + LagBasedAutoScalerConfig.class + ), + spec, + emitter + ); + + supervisor.start(); + autoScaler.start(); + supervisor.runInternal(); + + int before = supervisor.getIoConfig().getTaskCount(); + Assert.assertEquals(1, before); + Thread.sleep(1000); // allow one dynamic allocation cycle + int after = supervisor.getIoConfig().getTaskCount(); + // Even though metadata insert failed, taskCount should still be updated in ioConfig + Assert.assertEquals(2, after); + + autoScaler.reset(); + autoScaler.stop(); + } + @Test public void testMergeSpecConfigs() { From 299ee812370358a5bea5086249f14634742d9033 Mon Sep 17 00:00:00 2001 From: Sasha Syrotenko Date: Mon, 5 Jan 2026 12:51:03 +0200 Subject: [PATCH 20/20] Create a new java class for maybeScaleDuringTaskRollover method --- .../supervisor/SeekableStreamSupervisor.java | 5 +- ...SupervisorScaleDuringTaskRolloverTest.java | 612 ++++++++++++++++++ 2 files changed, 615 insertions(+), 2 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorScaleDuringTaskRolloverTest.java 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 0148f2225e09..c5d3ffe873d5 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 @@ -3444,7 +3444,7 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException activelyReadingTaskGroups.remove(groupId); } - maybeScaleUpDuringTaskRollover(); + maybeScaleDuringTaskRollover(); } /** @@ -3453,7 +3453,8 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException * This method is invoked to determine whether a task count adjustment is needed * during a task rollover based on the recommendations from the task auto-scaler. */ - private void maybeScaleUpDuringTaskRollover() + @VisibleForTesting + void maybeScaleDuringTaskRollover() { if (taskAutoScaler != null && activelyReadingTaskGroups.isEmpty()) { int rolloverTaskCount = taskAutoScaler.computeTaskCountForRollover(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorScaleDuringTaskRolloverTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorScaleDuringTaskRolloverTest.java new file mode 100644 index 000000000000..3e4bcd92b99f --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorScaleDuringTaskRolloverTest.java @@ -0,0 +1,612 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.seekablestream.supervisor; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; +import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; +import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScalerConfig; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.indexing.DataSchema; +import org.easymock.EasyMock; +import org.easymock.EasyMockSupport; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.File; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.ScheduledExecutorService; + +public class SeekableStreamSupervisorScaleDuringTaskRolloverTest extends EasyMockSupport +{ + private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); + private static final String STREAM = "stream"; + private static final String DATASOURCE = "testDS"; + private static final String SUPERVISOR = "supervisor"; + private static final int DEFAULT_TASK_COUNT = 10; + + private TaskStorage taskStorage; + private TaskMaster taskMaster; + private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private ServiceEmitter emitter; + private RowIngestionMetersFactory rowIngestionMetersFactory; + private SeekableStreamIndexTaskClientFactory taskClientFactory; + private SeekableStreamSupervisorSpec spec; + private SupervisorStateManagerConfig supervisorConfig; + + @Before + public void setUp() + { + taskStorage = EasyMock.mock(TaskStorage.class); + taskMaster = EasyMock.mock(TaskMaster.class); + indexerMetadataStorageCoordinator = EasyMock.mock(IndexerMetadataStorageCoordinator.class); + emitter = new StubServiceEmitter(); + rowIngestionMetersFactory = EasyMock.mock(RowIngestionMetersFactory.class); + taskClientFactory = EasyMock.mock(SeekableStreamIndexTaskClientFactory.class); + spec = EasyMock.mock(SeekableStreamSupervisorSpec.class); + supervisorConfig = new SupervisorStateManagerConfig(); + + // Common taskMaster setup - used by all tests + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.replay(taskMaster); + } + + @Test + public void test_maybeScaleDuringTaskRollover_noAutoScaler_doesNotScale() + { + // Given + setupSpecExpectations(getIOConfigWithoutAutoScaler(5)); + EasyMock.expect(spec.createAutoscaler(EasyMock.anyObject())).andReturn(null).anyTimes(); + EasyMock.replay(spec); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(10); + supervisor.start(); + + int beforeTaskCount = supervisor.getIoConfig().getTaskCount(); + + // When + supervisor.maybeScaleDuringTaskRollover(); + + // Then + Assert.assertEquals( + "Task count should not change when taskAutoScaler is null", + beforeTaskCount, + (int) supervisor.getIoConfig().getTaskCount() + ); + } + + @Test + public void test_maybeScaleDuringTaskRollover_rolloverCountNonPositive_doesNotScale() + { + // Given + setupSpecExpectations(getIOConfigWithCostBasedAutoScaler()); + EasyMock.expect(spec.createAutoscaler(EasyMock.anyObject())) + .andReturn(createMockAutoScaler(-1)) + .anyTimes(); + EasyMock.replay(spec); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(100); + supervisor.start(); + supervisor.createAutoscaler(spec); + + int beforeTaskCount = supervisor.getIoConfig().getTaskCount(); + + // When + supervisor.maybeScaleDuringTaskRollover(); + + // Then + Assert.assertEquals( + "Task count should not change when rolloverTaskCount <= 0", + beforeTaskCount, + (int) supervisor.getIoConfig().getTaskCount() + ); + } + + @Test + public void test_maybeScaleDuringTaskRollover_rolloverCountPositive_performsScaling() + { + // Given + final int targetTaskCount = 5; + + setupSpecExpectations(getIOConfigWithCostBasedAutoScaler()); + EasyMock.expect(spec.createAutoscaler(EasyMock.anyObject())) + .andReturn(createMockAutoScaler(targetTaskCount)) + .anyTimes(); + EasyMock.replay(spec); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(100); + supervisor.start(); + supervisor.createAutoscaler(spec); + + Assert.assertEquals(1, (int) supervisor.getIoConfig().getTaskCount()); + + // When + supervisor.maybeScaleDuringTaskRollover(); + + // Then + Assert.assertEquals( + "Task count should be updated to " + targetTaskCount + " when rolloverTaskCount > 0", + targetTaskCount, + (int) supervisor.getIoConfig().getTaskCount() + ); + } + + @Test + public void test_maybeScaleDuringTaskRollover_rolloverCountZero_doesNotScale() + { + // Given + setupSpecExpectations(getIOConfigWithCostBasedAutoScaler()); + EasyMock.expect(spec.createAutoscaler(EasyMock.anyObject())) + .andReturn(createMockAutoScaler(0)) + .anyTimes(); + EasyMock.replay(spec); + + TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(100); + supervisor.start(); + supervisor.createAutoscaler(spec); + + int beforeTaskCount = supervisor.getIoConfig().getTaskCount(); + + // When + supervisor.maybeScaleDuringTaskRollover(); + + // Then + Assert.assertEquals( + "Task count should not change when rolloverTaskCount is 0", + beforeTaskCount, + (int) supervisor.getIoConfig().getTaskCount() + ); + } + + // Helper methods for test setup + + /** + * Sets up common spec expectations. Call EasyMock.replay(spec) after this and any additional expectations. + */ + private void setupSpecExpectations(SeekableStreamSupervisorIOConfig ioConfig) + { + EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); + EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); + EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); + EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + } + + /** + * Creates a mock autoscaler that returns the specified rollover count. + */ + private static SupervisorTaskAutoScaler createMockAutoScaler(int rolloverCount) + { + return new SupervisorTaskAutoScaler() + { + @Override + public void start() + { + } + + @Override + public void stop() + { + } + + @Override + public void reset() + { + } + + @Override + public int computeTaskCountForRollover() + { + return rolloverCount; + } + }; + } + + // Helper methods for config creation + + private static CostBasedAutoScalerConfig getCostBasedAutoScalerConfig() + { + return CostBasedAutoScalerConfig.builder() + .taskCountMax(100) + .taskCountMin(1) + .enableTaskAutoScaler(true) + .lagWeight(0.3) + .idleWeight(0.7) + .scaleActionPeriodMillis(100) + .build(); + } + + private SeekableStreamSupervisorIOConfig getIOConfigWithCostBasedAutoScaler() + { + return createIOConfig(DEFAULT_TASK_COUNT, getCostBasedAutoScalerConfig()); + } + + private SeekableStreamSupervisorIOConfig getIOConfigWithoutAutoScaler(int taskCount) + { + return createIOConfig(taskCount, null); + } + + private SeekableStreamSupervisorIOConfig createIOConfig(int taskCount, CostBasedAutoScalerConfig autoScalerConfig) + { + return new SeekableStreamSupervisorIOConfig( + STREAM, + new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), false, false, false), + 1, + taskCount, + new Period("PT1H"), + new Period("P1D"), + new Period("PT30S"), + false, + new Period("PT30M"), + null, + null, + autoScalerConfig, + LagAggregator.DEFAULT, + null, + null, + null + ) + { + }; + } + + private static DataSchema getDataSchema() + { + List dimensions = new ArrayList<>(); + dimensions.add(StringDimensionSchema.create("dim1")); + dimensions.add(StringDimensionSchema.create("dim2")); + + return DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); + } + + private static SeekableStreamSupervisorTuningConfig getTuningConfig() + { + return new SeekableStreamSupervisorTuningConfig() + { + @Override + public Integer getWorkerThreads() + { + return 1; + } + + @Override + public Long getChatRetries() + { + return 1L; + } + + @Override + public Duration getHttpTimeout() + { + return new Period("PT1M").toStandardDuration(); + } + + @Override + public Duration getShutdownTimeout() + { + return new Period("PT1S").toStandardDuration(); + } + + @Override + public Duration getRepartitionTransitionDuration() + { + return new Period("PT2M").toStandardDuration(); + } + + @Override + public Duration getOffsetFetchPeriod() + { + return new Period("PT5M").toStandardDuration(); + } + + @Override + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + { + return new SeekableStreamIndexTaskTuningConfig(null, null, null, null, null, null, null, null, null, null, + null, null, null, null, null, null, null, null, null, null, + null, null, null + ) + { + @Override + public SeekableStreamIndexTaskTuningConfig withBasePersistDirectory(File dir) + { + return null; + } + + @Override + public String toString() + { + return null; + } + }; + } + }; + } + + // Inner test classes + + private abstract class BaseTestSeekableStreamSupervisor extends SeekableStreamSupervisor + { + private BaseTestSeekableStreamSupervisor() + { + super( + "testSupervisorId", + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + spec, + rowIngestionMetersFactory, + false + ); + } + + @Override + protected String baseTaskName() + { + return "test"; + } + + @Override + protected void updatePartitionLagFromStream() + { + } + + @Nullable + @Override + protected Map getPartitionRecordLag() + { + return null; + } + + @Nullable + @Override + protected Map getPartitionTimeLag() + { + return null; + } + + @Override + protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( + int groupId, + Map startPartitions, + Map endPartitions, + String baseSequenceName, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + SeekableStreamSupervisorIOConfig ioConfig + ) + { + return new SeekableStreamIndexTaskIOConfig<>( + groupId, + baseSequenceName, + new SeekableStreamStartSequenceNumbers<>(STREAM, startPartitions, exclusiveStartSequenceNumberPartitions), + new SeekableStreamEndSequenceNumbers<>(STREAM, endPartitions), + true, + minimumMessageTime, + maximumMessageTime, + ioConfig.getInputFormat(), + ioConfig.getTaskDuration().getStandardMinutes() + ) + { + }; + } + + @Override + protected List> createIndexTasks( + int replicas, + String baseSequenceName, + ObjectMapper sortingMapper, + TreeMap> sequenceOffsets, + SeekableStreamIndexTaskIOConfig taskIoConfig, + SeekableStreamIndexTaskTuningConfig taskTuningConfig, + RowIngestionMetersFactory rowIngestionMetersFactory + ) + { + return null; + } + + @Override + protected int getTaskGroupIdForPartition(String partition) + { + return 0; + } + + @Override + protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) + { + return true; + } + + @Override + protected boolean doesTaskMatchSupervisor(Task task) + { + return true; + } + + @Override + protected SeekableStreamDataSourceMetadata createDataSourceMetaDataForReset( + String stream, + Map map + ) + { + return null; + } + + @Override + protected OrderedSequenceNumber makeSequenceNumber(String seq, boolean isExclusive) + { + return new OrderedSequenceNumber<>(seq, isExclusive) + { + @Override + public int compareTo(OrderedSequenceNumber o) + { + return new BigInteger(this.get()).compareTo(new BigInteger(o.get())); + } + }; + } + + @Override + protected Map getRecordLagPerPartition(Map currentOffsets) + { + return null; + } + + @Override + protected Map getTimeLagPerPartition(Map currentOffsets) + { + return null; + } + + @Override + protected RecordSupplier setupRecordSupplier() + { + return recordSupplier; + } + + @Override + protected SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, + boolean includeOffsets + ) + { + return new SeekableStreamSupervisorReportPayload<>(SUPERVISOR, DATASOURCE, STREAM, 1, 1, 1L, + null, null, null, null, null, null, + false, true, null, null, null + ) + { + }; + } + + @Override + protected String getNotSetMarker() + { + return "NOT_SET"; + } + + @Override + protected String getEndOfPartitionMarker() + { + return "EOF"; + } + + @Override + protected boolean isEndOfShard(String seqNum) + { + return false; + } + + @Override + protected boolean isShardExpirationMarker(String seqNum) + { + return false; + } + + @Override + protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() + { + return false; + } + } + + private class TestSeekableStreamSupervisor extends BaseTestSeekableStreamSupervisor + { + private final int partitionNumbers; + + public TestSeekableStreamSupervisor(int partitionNumbers) + { + this.partitionNumbers = partitionNumbers; + } + + @Override + protected void scheduleReporting(ScheduledExecutorService reportingExec) + { + } + + @Override + public LagStats computeLagStats() + { + return new LagStats(0, 0, 0); + } + + @Override + public int getPartitionCount() + { + return partitionNumbers; + } + } +}