kfaraz commented on code in PR #18819:
URL: https://github.com/apache/druid/pull/18819#discussion_r2618214615
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java:
##########
@@ -1662,6 +1664,9 @@ public Map<String, Object> doGetRowStats()
returnMap.put("movingAverages", averagesMap);
returnMap.put("totals", totalsMap);
+ if (this.recordSupplier != null) {
+ returnMap.put("pollIdleRatio",
this.recordSupplier.getPollIdleRatioMetric());
Review Comment:
Let's not put this metric inside top-level map.
It might be better to put this inside another top-level map (at the same
level as `totals` and `movingAverages`) named something like `consumerStats` or
`autoScalerStats`.
Also, please add a query parameter `autoScalerStatsOnly` (false by default)
to the `/rowStats` API so that when the auto-scaler invokes this API, it only
gets back the metrics that it needs. When the query param is false, we would
send back all the stats.
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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;
+
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
+import
org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
+import
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.query.DruidMetrics;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Cost-based auto-scaler for seekable stream supervisors.
+ * Uses a cost function combining lag and idle time metrics to determine
optimal task counts.
+ * Task counts are selected from pre-calculated values (not arbitrary factors).
+ * Scale-up happens incrementally, scale-down only during task rollover.
+ */
+public class CostBasedAutoScaler implements SupervisorTaskAutoScaler
+{
+ private static final EmittingLogger log = new
EmittingLogger(CostBasedAutoScaler.class);
+
+ private static final int SCALE_FACTOR_DISCRETE_DISTANCE = 2;
+ public static final String OPTIMAL_TASK_COUNT_METRIC =
"task/autoScaler/costBased/optimalTaskCount";
+
+ private final String supervisorId;
+ private final SeekableStreamSupervisor supervisor;
+ private final ServiceEmitter emitter;
+ private final SupervisorSpec spec;
+ private final CostBasedAutoScalerConfig config;
+ private final ServiceMetricEvent.Builder metricBuilder;
+ /**
+ * Atomic reference to CostMetrics object. All operations must be performed
+ * with sequentially consistent semantics (volatile reads/writes).
+ * However, it may be fine-tuned with acquire/release semantics,
+ * but requires careful reasoning about correctness.
+ */
+ private final AtomicReference<CostMetrics> currentMetrics;
+ private final ScheduledExecutorService autoscalerExecutor;
+ private final WeightedCostFunction costFunction;
+
+ public CostBasedAutoScaler(
+ SeekableStreamSupervisor supervisor,
+ CostBasedAutoScalerConfig config,
+ SupervisorSpec spec,
+ ServiceEmitter emitter
+ )
+ {
+ this.config = config;
+ this.spec = spec;
+ this.supervisor = supervisor;
+ this.supervisorId = spec.getId();
+ this.emitter = emitter;
+
+ this.currentMetrics = new AtomicReference<>(null);
+ this.costFunction = new WeightedCostFunction();
+
+ this.autoscalerExecutor =
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+ this.metricBuilder = ServiceMetricEvent.builder()
+
.setDimension(DruidMetrics.DATASOURCE, supervisorId)
+ .setDimension(
+ DruidMetrics.STREAM,
+
this.supervisor.getIoConfig().getStream()
+ );
+ }
+
+ @Override
+ public void start()
+ {
+ Callable<Integer> scaleAction = () ->
computeOptimalTaskCount(currentMetrics);
+ Runnable onSuccessfulScale = () -> currentMetrics.set(null);
+
+ autoscalerExecutor.scheduleAtFixedRate(
+ this::collectMetrics,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getMetricsCollectionIntervalMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ autoscalerExecutor.scheduleAtFixedRate(
+ supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale,
emitter),
+ config.getScaleActionStartDelayMillis(),
+ config.getScaleActionPeriodMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ log.info(
+ "CostBasedAutoScaler started for dataSource [%s]: collecting metrics
every [%d]ms, "
+ + "evaluating scaling every [%d]ms",
+ supervisorId,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getScaleActionPeriodMillis()
+ );
+ }
+
+ @Override
+ public void stop()
+ {
+ autoscalerExecutor.shutdownNow();
+ log.info("CostBasedAutoScaler stopped for dataSource [%s]", supervisorId);
+ }
+
+ @Override
+ public void reset()
+ {
+ currentMetrics.set(null);
+ }
+
+ private void collectMetrics()
+ {
+ if (spec.isSuspended()) {
+ log.debug("Supervisor [%s] is suspended, skipping a metrics collection",
supervisorId);
+ return;
+ }
+
+ final LagStats lagStats = supervisor.computeLagStats();
+ if (lagStats == null) {
+ log.debug("Lag stats unavailable for dataSource [%s], skipping
collection", supervisorId);
+ return;
+ }
+
+ final int currentTaskCount = supervisor.getIoConfig().getTaskCount();
+ final int partitionCount = supervisor.getPartitionCount();
+ final double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+ currentMetrics.set(
+ new CostMetrics(
+ lagStats.getAvgLag(),
+ currentTaskCount,
+ partitionCount,
+ pollIdleRatio
+ )
+ );
+
+ log.debug("Collected metrics for dataSource [%s]", supervisorId);
+ }
+
+ /**
+ * Computes the optimal task count based on current metrics.
+ * <p>
+ * Returns -1 (no scaling needed) in the following cases:
+ * <ul>
+ * <li>Metrics are not available</li>
+ * <li>The current idle ratio is in the ideal range [0.2, 0.6] - optimal
utilization achieved</li>
+ * <li>Optimal task count equals current task count</li>
+ * </ul>
+ *
+ * @return optimal task count for scale-up, or -1 if no scaling action needed
+ */
+ public int computeOptimalTaskCount(AtomicReference<CostMetrics>
currentMetricsRef)
+ {
+ final CostMetrics metrics = currentMetricsRef.get();
+ if (metrics == null) {
+ log.debug("No metrics available yet for dataSource [%s]", supervisorId);
+ return -1;
+ }
+
+ final int partitionCount = metrics.getPartitionCount();
+ final int currentTaskCount = metrics.getCurrentTaskCount();
+ if (partitionCount <= 0 || currentTaskCount <= 0) {
+ return -1;
+ }
+
+ final int[] validTaskCounts =
CostBasedAutoScaler.computeFactors(partitionCount);
+
+ if (validTaskCounts.length == 0) {
+ log.warn("No valid task counts after applying constraints for dataSource
[%s]", supervisorId);
+ 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 current task
count.
+ final double currentIdleRatio = metrics.getPollIdleRatio();
+ if (currentIdleRatio >= 0 &&
WeightedCostFunction.isIdleInIdealRange(currentIdleRatio)) {
+ log.info(
+ "Idle ratio [%.3f] is in ideal range for dataSource [%s], no scaling
needed",
+ currentIdleRatio,
+ supervisorId
+ );
+ return -1;
+ }
+
+ // Update bounds with observed lag BEFORE optimization loop
+ // This ensures normalization uses historical observed values, not
predicted values
+ costFunction.updateLagBounds(metrics.getAvgPartitionLag());
+
+ int optimalTaskCount = -1;
+ double optimalCost = Double.POSITIVE_INFINITY;
+
+ final int bestTaskCountIndex = Arrays.binarySearch(validTaskCounts,
currentTaskCount);
+ for (int i = bestTaskCountIndex - SCALE_FACTOR_DISCRETE_DISTANCE;
+ i <= bestTaskCountIndex + SCALE_FACTOR_DISCRETE_DISTANCE; i++) {
+ // Range check.
+ if (i < 0 || i >= validTaskCounts.length) {
+ continue;
+ }
+ int taskCount = validTaskCounts[i];
+ if (taskCount < config.getTaskCountMin()) {
+ continue;
+ } else if (taskCount > config.getTaskCountMax()) {
+ break;
+ }
+ double cost = costFunction.computeCost(metrics, taskCount, config);
+ log.debug("Proposed task count: %d, Cost: %.4f", taskCount, cost);
+ if (cost < optimalCost) {
+ optimalTaskCount = taskCount;
+ optimalCost = cost;
+ }
+ }
+
+ emitter.emit(metricBuilder.setMetric(OPTIMAL_TASK_COUNT_METRIC, (long)
optimalTaskCount));
+
+ log.info(
+ "Cost-based scaling evaluation for dataSource [%s]: current=%d,
optimal=%d, cost=%.4f, "
+ + "avgPartitionLag=%.2f, pollIdleRatio=%.3f",
+ supervisorId,
+ metrics.getCurrentTaskCount(),
+ optimalTaskCount,
+ optimalCost,
+ metrics.getAvgPartitionLag(),
+ metrics.getPollIdleRatio()
+ );
+
+ if (optimalTaskCount > currentTaskCount) {
+ return optimalTaskCount;
+ } else if (optimalTaskCount < currentTaskCount) {
+ supervisor.getIoConfig().setTaskCount(optimalTaskCount);
+ }
+ return -1;
+ }
+
+ /**
+ * Generates valid task counts based on partitions-per-task ratios.
+ * This enables gradual scaling and avoids large jumps.
+ *
+ * @return sorted list of valid task counts within bounds
+ */
+ static int[] computeFactors(int partitionCount)
Review Comment:
```suggestion
static int[] computeValidTaskCounts(int partitionCount)
```
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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;
+
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
+import
org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
+import
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.query.DruidMetrics;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Cost-based auto-scaler for seekable stream supervisors.
+ * Uses a cost function combining lag and idle time metrics to determine
optimal task counts.
+ * Task counts are selected from pre-calculated values (not arbitrary factors).
+ * Scale-up happens incrementally, scale-down only during task rollover.
+ */
+public class CostBasedAutoScaler implements SupervisorTaskAutoScaler
+{
+ private static final EmittingLogger log = new
EmittingLogger(CostBasedAutoScaler.class);
+
+ private static final int SCALE_FACTOR_DISCRETE_DISTANCE = 2;
+ public static final String OPTIMAL_TASK_COUNT_METRIC =
"task/autoScaler/costBased/optimalTaskCount";
+
+ private final String supervisorId;
+ private final SeekableStreamSupervisor supervisor;
+ private final ServiceEmitter emitter;
+ private final SupervisorSpec spec;
+ private final CostBasedAutoScalerConfig config;
+ private final ServiceMetricEvent.Builder metricBuilder;
+ /**
+ * Atomic reference to CostMetrics object. All operations must be performed
+ * with sequentially consistent semantics (volatile reads/writes).
+ * However, it may be fine-tuned with acquire/release semantics,
+ * but requires careful reasoning about correctness.
+ */
+ private final AtomicReference<CostMetrics> currentMetrics;
+ private final ScheduledExecutorService autoscalerExecutor;
+ private final WeightedCostFunction costFunction;
+
+ public CostBasedAutoScaler(
+ SeekableStreamSupervisor supervisor,
+ CostBasedAutoScalerConfig config,
+ SupervisorSpec spec,
+ ServiceEmitter emitter
+ )
+ {
+ this.config = config;
+ this.spec = spec;
+ this.supervisor = supervisor;
+ this.supervisorId = spec.getId();
+ this.emitter = emitter;
+
+ this.currentMetrics = new AtomicReference<>(null);
+ this.costFunction = new WeightedCostFunction();
+
+ this.autoscalerExecutor =
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+ this.metricBuilder = ServiceMetricEvent.builder()
+
.setDimension(DruidMetrics.DATASOURCE, supervisorId)
+ .setDimension(
+ DruidMetrics.STREAM,
+
this.supervisor.getIoConfig().getStream()
+ );
+ }
+
+ @Override
+ public void start()
+ {
+ Callable<Integer> scaleAction = () ->
computeOptimalTaskCount(currentMetrics);
+ Runnable onSuccessfulScale = () -> currentMetrics.set(null);
+
+ autoscalerExecutor.scheduleAtFixedRate(
+ this::collectMetrics,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getMetricsCollectionIntervalMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ autoscalerExecutor.scheduleAtFixedRate(
+ supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale,
emitter),
+ config.getScaleActionStartDelayMillis(),
+ config.getScaleActionPeriodMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ log.info(
+ "CostBasedAutoScaler started for dataSource [%s]: collecting metrics
every [%d]ms, "
+ + "evaluating scaling every [%d]ms",
+ supervisorId,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getScaleActionPeriodMillis()
+ );
+ }
+
+ @Override
+ public void stop()
+ {
+ autoscalerExecutor.shutdownNow();
+ log.info("CostBasedAutoScaler stopped for dataSource [%s]", supervisorId);
+ }
+
+ @Override
+ public void reset()
+ {
+ currentMetrics.set(null);
+ }
+
+ private void collectMetrics()
+ {
+ if (spec.isSuspended()) {
+ log.debug("Supervisor [%s] is suspended, skipping a metrics collection",
supervisorId);
+ return;
+ }
+
+ final LagStats lagStats = supervisor.computeLagStats();
+ if (lagStats == null) {
+ log.debug("Lag stats unavailable for dataSource [%s], skipping
collection", supervisorId);
+ return;
+ }
+
+ final int currentTaskCount = supervisor.getIoConfig().getTaskCount();
+ final int partitionCount = supervisor.getPartitionCount();
+ final double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+ currentMetrics.set(
+ new CostMetrics(
+ lagStats.getAvgLag(),
+ currentTaskCount,
+ partitionCount,
+ pollIdleRatio
+ )
+ );
+
+ log.debug("Collected metrics for dataSource [%s]", supervisorId);
+ }
+
+ /**
+ * Computes the optimal task count based on current metrics.
+ * <p>
+ * Returns -1 (no scaling needed) in the following cases:
+ * <ul>
+ * <li>Metrics are not available</li>
+ * <li>The current idle ratio is in the ideal range [0.2, 0.6] - optimal
utilization achieved</li>
+ * <li>Optimal task count equals current task count</li>
+ * </ul>
+ *
+ * @return optimal task count for scale-up, or -1 if no scaling action needed
+ */
+ public int computeOptimalTaskCount(AtomicReference<CostMetrics>
currentMetricsRef)
+ {
+ final CostMetrics metrics = currentMetricsRef.get();
+ if (metrics == null) {
+ log.debug("No metrics available yet for dataSource [%s]", supervisorId);
+ return -1;
+ }
+
+ final int partitionCount = metrics.getPartitionCount();
+ final int currentTaskCount = metrics.getCurrentTaskCount();
+ if (partitionCount <= 0 || currentTaskCount <= 0) {
+ return -1;
+ }
+
+ final int[] validTaskCounts =
CostBasedAutoScaler.computeFactors(partitionCount);
+
+ if (validTaskCounts.length == 0) {
+ log.warn("No valid task counts after applying constraints for dataSource
[%s]", supervisorId);
+ 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 current task
count.
+ final double currentIdleRatio = metrics.getPollIdleRatio();
+ if (currentIdleRatio >= 0 &&
WeightedCostFunction.isIdleInIdealRange(currentIdleRatio)) {
+ log.info(
+ "Idle ratio [%.3f] is in ideal range for dataSource [%s], no scaling
needed",
+ currentIdleRatio,
+ supervisorId
+ );
+ return -1;
+ }
+
+ // Update bounds with observed lag BEFORE optimization loop
+ // This ensures normalization uses historical observed values, not
predicted values
+ costFunction.updateLagBounds(metrics.getAvgPartitionLag());
+
+ int optimalTaskCount = -1;
+ double optimalCost = Double.POSITIVE_INFINITY;
+
+ final int bestTaskCountIndex = Arrays.binarySearch(validTaskCounts,
currentTaskCount);
+ for (int i = bestTaskCountIndex - SCALE_FACTOR_DISCRETE_DISTANCE;
+ i <= bestTaskCountIndex + SCALE_FACTOR_DISCRETE_DISTANCE; i++) {
+ // Range check.
+ if (i < 0 || i >= validTaskCounts.length) {
+ continue;
+ }
+ int taskCount = validTaskCounts[i];
+ if (taskCount < config.getTaskCountMin()) {
+ continue;
+ } else if (taskCount > config.getTaskCountMax()) {
+ break;
+ }
+ double cost = costFunction.computeCost(metrics, taskCount, config);
+ log.debug("Proposed task count: %d, Cost: %.4f", taskCount, cost);
+ if (cost < optimalCost) {
+ optimalTaskCount = taskCount;
+ optimalCost = cost;
+ }
+ }
+
+ emitter.emit(metricBuilder.setMetric(OPTIMAL_TASK_COUNT_METRIC, (long)
optimalTaskCount));
+
+ log.info(
+ "Cost-based scaling evaluation for dataSource [%s]: current=%d,
optimal=%d, cost=%.4f, "
+ + "avgPartitionLag=%.2f, pollIdleRatio=%.3f",
+ supervisorId,
+ metrics.getCurrentTaskCount(),
+ optimalTaskCount,
+ optimalCost,
+ metrics.getAvgPartitionLag(),
+ metrics.getPollIdleRatio()
+ );
+
+ if (optimalTaskCount > currentTaskCount) {
+ return optimalTaskCount;
+ } else if (optimalTaskCount < currentTaskCount) {
+ supervisor.getIoConfig().setTaskCount(optimalTaskCount);
+ }
+ return -1;
+ }
+
+ /**
+ * Generates valid task counts based on partitions-per-task ratios.
+ * This enables gradual scaling and avoids large jumps.
+ *
+ * @return sorted list of valid task counts within bounds
+ */
+ static int[] computeFactors(int partitionCount)
+ {
+ if (partitionCount <= 0) {
+ return new int[]{};
+ }
+
+ List<Integer> result = new ArrayList<>();
+
+ for (int partitionsPerTask = partitionCount; partitionsPerTask >= 1;
partitionsPerTask--) {
Review Comment:
We shouldn't be computing all possible valid task counts if we are only ever
going to use the task counts in the +2/-2 window.
Please try to simplify this computation to something similar to the one
suggested in https://github.com/apache/druid/pull/18819#discussion_r2605256520.
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java:
##########
@@ -4381,6 +4371,36 @@ public ConcurrentHashMap<PartitionIdType,
SequenceOffsetType> getPartitionOffset
return partitionOffsets;
}
+ /**
+ * Calculates the average poll-idle-ratio metric across all active tasks.
+ * This metric indicates how much time the consumer spends idle waiting for
data.
+ *
+ * @return the average poll-idle-ratio across all tasks, or 1 (full idle) if
no tasks or metrics are available
+ */
+ public double getPollIdleRatioMetric()
+ {
+ Map<String, Map<String, Object>> taskMetrics = getStats();
+ if (taskMetrics.isEmpty()) {
+ return 1.;
+ }
+
+ double sum = 0;
+ int count = 0;
+ for (Map<String, Object> groupMetrics : taskMetrics.values()) {
+ for (Object taskMetric : groupMetrics.values()) {
+ if (taskMetric instanceof Map) {
+ Object pollIdleRatio = ((Map<?, ?>) taskMetric).get("pollIdleRatio");
Review Comment:
Might be nice to have this string as a constant.
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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;
+
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
+import
org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
+import
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.query.DruidMetrics;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Cost-based auto-scaler for seekable stream supervisors.
+ * Uses a cost function combining lag and idle time metrics to determine
optimal task counts.
+ * Task counts are selected from pre-calculated values (not arbitrary factors).
+ * Scale-up happens incrementally, scale-down only during task rollover.
+ */
+public class CostBasedAutoScaler implements SupervisorTaskAutoScaler
+{
+ private static final EmittingLogger log = new
EmittingLogger(CostBasedAutoScaler.class);
+
+ private static final int SCALE_FACTOR_DISCRETE_DISTANCE = 2;
+ public static final String OPTIMAL_TASK_COUNT_METRIC =
"task/autoScaler/costBased/optimalTaskCount";
+
+ private final String supervisorId;
+ private final SeekableStreamSupervisor supervisor;
+ private final ServiceEmitter emitter;
+ private final SupervisorSpec spec;
+ private final CostBasedAutoScalerConfig config;
+ private final ServiceMetricEvent.Builder metricBuilder;
+ /**
+ * Atomic reference to CostMetrics object. All operations must be performed
+ * with sequentially consistent semantics (volatile reads/writes).
+ * However, it may be fine-tuned with acquire/release semantics,
+ * but requires careful reasoning about correctness.
+ */
+ private final AtomicReference<CostMetrics> currentMetrics;
+ private final ScheduledExecutorService autoscalerExecutor;
+ private final WeightedCostFunction costFunction;
+
+ public CostBasedAutoScaler(
+ SeekableStreamSupervisor supervisor,
+ CostBasedAutoScalerConfig config,
+ SupervisorSpec spec,
+ ServiceEmitter emitter
+ )
+ {
+ this.config = config;
+ this.spec = spec;
+ this.supervisor = supervisor;
+ this.supervisorId = spec.getId();
+ this.emitter = emitter;
+
+ this.currentMetrics = new AtomicReference<>(null);
+ this.costFunction = new WeightedCostFunction();
+
+ this.autoscalerExecutor =
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+ this.metricBuilder = ServiceMetricEvent.builder()
+
.setDimension(DruidMetrics.DATASOURCE, supervisorId)
+ .setDimension(
+ DruidMetrics.STREAM,
+
this.supervisor.getIoConfig().getStream()
+ );
+ }
+
+ @Override
+ public void start()
+ {
+ Callable<Integer> scaleAction = () ->
computeOptimalTaskCount(currentMetrics);
+ Runnable onSuccessfulScale = () -> currentMetrics.set(null);
+
+ autoscalerExecutor.scheduleAtFixedRate(
+ this::collectMetrics,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getMetricsCollectionIntervalMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ autoscalerExecutor.scheduleAtFixedRate(
+ supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale,
emitter),
+ config.getScaleActionStartDelayMillis(),
+ config.getScaleActionPeriodMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ log.info(
+ "CostBasedAutoScaler started for dataSource [%s]: collecting metrics
every [%d]ms, "
+ + "evaluating scaling every [%d]ms",
+ supervisorId,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getScaleActionPeriodMillis()
+ );
+ }
+
+ @Override
+ public void stop()
+ {
+ autoscalerExecutor.shutdownNow();
+ log.info("CostBasedAutoScaler stopped for dataSource [%s]", supervisorId);
+ }
+
+ @Override
+ public void reset()
+ {
+ currentMetrics.set(null);
+ }
+
+ private void collectMetrics()
+ {
+ if (spec.isSuspended()) {
+ log.debug("Supervisor [%s] is suspended, skipping a metrics collection",
supervisorId);
+ return;
+ }
+
+ final LagStats lagStats = supervisor.computeLagStats();
+ if (lagStats == null) {
+ log.debug("Lag stats unavailable for dataSource [%s], skipping
collection", supervisorId);
+ return;
+ }
+
+ final int currentTaskCount = supervisor.getIoConfig().getTaskCount();
+ final int partitionCount = supervisor.getPartitionCount();
+ final double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+ currentMetrics.set(
+ new CostMetrics(
+ lagStats.getAvgLag(),
+ currentTaskCount,
+ partitionCount,
+ pollIdleRatio
+ )
+ );
+
+ log.debug("Collected metrics for dataSource [%s]", supervisorId);
+ }
+
+ /**
+ * Computes the optimal task count based on current metrics.
+ * <p>
+ * Returns -1 (no scaling needed) in the following cases:
+ * <ul>
+ * <li>Metrics are not available</li>
+ * <li>The current idle ratio is in the ideal range [0.2, 0.6] - optimal
utilization achieved</li>
+ * <li>Optimal task count equals current task count</li>
+ * </ul>
+ *
+ * @return optimal task count for scale-up, or -1 if no scaling action needed
+ */
+ public int computeOptimalTaskCount(AtomicReference<CostMetrics>
currentMetricsRef)
+ {
+ final CostMetrics metrics = currentMetricsRef.get();
+ if (metrics == null) {
+ log.debug("No metrics available yet for dataSource [%s]", supervisorId);
+ return -1;
+ }
+
+ final int partitionCount = metrics.getPartitionCount();
+ final int currentTaskCount = metrics.getCurrentTaskCount();
+ if (partitionCount <= 0 || currentTaskCount <= 0) {
+ return -1;
+ }
+
+ final int[] validTaskCounts =
CostBasedAutoScaler.computeFactors(partitionCount);
+
+ if (validTaskCounts.length == 0) {
+ log.warn("No valid task counts after applying constraints for dataSource
[%s]", supervisorId);
+ 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 current task
count.
+ final double currentIdleRatio = metrics.getPollIdleRatio();
+ if (currentIdleRatio >= 0 &&
WeightedCostFunction.isIdleInIdealRange(currentIdleRatio)) {
+ log.info(
+ "Idle ratio [%.3f] is in ideal range for dataSource [%s], no scaling
needed",
+ currentIdleRatio,
+ supervisorId
+ );
+ return -1;
+ }
+
+ // Update bounds with observed lag BEFORE optimization loop
+ // This ensures normalization uses historical observed values, not
predicted values
+ costFunction.updateLagBounds(metrics.getAvgPartitionLag());
+
+ int optimalTaskCount = -1;
+ double optimalCost = Double.POSITIVE_INFINITY;
+
+ final int bestTaskCountIndex = Arrays.binarySearch(validTaskCounts,
currentTaskCount);
+ for (int i = bestTaskCountIndex - SCALE_FACTOR_DISCRETE_DISTANCE;
+ i <= bestTaskCountIndex + SCALE_FACTOR_DISCRETE_DISTANCE; i++) {
+ // Range check.
+ if (i < 0 || i >= validTaskCounts.length) {
+ continue;
+ }
+ int taskCount = validTaskCounts[i];
+ if (taskCount < config.getTaskCountMin()) {
+ continue;
+ } else if (taskCount > config.getTaskCountMax()) {
+ break;
+ }
+ double cost = costFunction.computeCost(metrics, taskCount, config);
+ log.debug("Proposed task count: %d, Cost: %.4f", taskCount, cost);
+ if (cost < optimalCost) {
+ optimalTaskCount = taskCount;
+ optimalCost = cost;
+ }
+ }
+
+ emitter.emit(metricBuilder.setMetric(OPTIMAL_TASK_COUNT_METRIC, (long)
optimalTaskCount));
+
+ log.info(
+ "Cost-based scaling evaluation for dataSource [%s]: current=%d,
optimal=%d, cost=%.4f, "
+ + "avgPartitionLag=%.2f, pollIdleRatio=%.3f",
+ supervisorId,
+ metrics.getCurrentTaskCount(),
+ optimalTaskCount,
+ optimalCost,
+ metrics.getAvgPartitionLag(),
+ metrics.getPollIdleRatio()
+ );
+
+ if (optimalTaskCount > currentTaskCount) {
+ return optimalTaskCount;
+ } else if (optimalTaskCount < currentTaskCount) {
+ supervisor.getIoConfig().setTaskCount(optimalTaskCount);
Review Comment:
This line can have behavioural side effects in the supervisor, since the
`taskCount` should always reflect the current running task count and not the
desired task count. Here we are updating the `taskCount` without actually
changing the number of tasks, or suspending the supervisor.
Instead, we could do the following:
- Add an auto-scaler method `isScaleDownOnRolloverOnly()`. This will always
return false for lag-based and always true for cost-based.
- `CostBasedAutoScalerConfig.computeOptimalTaskCount()` should return the
optimal task count for scale down cases as well.
- `SeekableStreamSupervisor` can store this desired task count in an atomic
boolean and retrieve it upon regular task rollover.
Another (perhaps cleaner) option is to simply invoke
`AutoScaler.computeOptimalTaskCount()` whenever we do rollover and then just go
with the optimal task count.
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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;
+
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
+import
org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
+import
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.query.DruidMetrics;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Cost-based auto-scaler for seekable stream supervisors.
+ * Uses a cost function combining lag and idle time metrics to determine
optimal task counts.
+ * Task counts are selected from pre-calculated values (not arbitrary factors).
+ * Scale-up happens incrementally, scale-down only during task rollover.
+ */
+public class CostBasedAutoScaler implements SupervisorTaskAutoScaler
+{
+ private static final EmittingLogger log = new
EmittingLogger(CostBasedAutoScaler.class);
+
+ private static final int SCALE_FACTOR_DISCRETE_DISTANCE = 2;
+ public static final String OPTIMAL_TASK_COUNT_METRIC =
"task/autoScaler/costBased/optimalTaskCount";
+
+ private final String supervisorId;
+ private final SeekableStreamSupervisor supervisor;
+ private final ServiceEmitter emitter;
+ private final SupervisorSpec spec;
+ private final CostBasedAutoScalerConfig config;
+ private final ServiceMetricEvent.Builder metricBuilder;
+ /**
+ * Atomic reference to CostMetrics object. All operations must be performed
+ * with sequentially consistent semantics (volatile reads/writes).
+ * However, it may be fine-tuned with acquire/release semantics,
+ * but requires careful reasoning about correctness.
+ */
+ private final AtomicReference<CostMetrics> currentMetrics;
+ private final ScheduledExecutorService autoscalerExecutor;
+ private final WeightedCostFunction costFunction;
+
+ public CostBasedAutoScaler(
+ SeekableStreamSupervisor supervisor,
+ CostBasedAutoScalerConfig config,
+ SupervisorSpec spec,
+ ServiceEmitter emitter
+ )
+ {
+ this.config = config;
+ this.spec = spec;
+ this.supervisor = supervisor;
+ this.supervisorId = spec.getId();
+ this.emitter = emitter;
+
+ this.currentMetrics = new AtomicReference<>(null);
+ this.costFunction = new WeightedCostFunction();
+
+ this.autoscalerExecutor =
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+ this.metricBuilder = ServiceMetricEvent.builder()
+
.setDimension(DruidMetrics.DATASOURCE, supervisorId)
Review Comment:
```suggestion
.setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId)
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]