kfaraz commented on code in PR #18819:
URL: https://github.com/apache/druid/pull/18819#discussion_r2619689343


##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScalerConfig.java:
##########
@@ -69,8 +65,6 @@ public CostBasedAutoScalerConfig(
       @Nullable @JsonProperty("taskCountStart") Integer taskCountStart,
       @Nullable @JsonProperty("minTriggerScaleActionFrequencyMillis") Long 
minTriggerScaleActionFrequencyMillis,

Review Comment:
   Please remove this field if it is not used anymore.



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java:
##########
@@ -134,6 +134,15 @@ ListenableFuture<Boolean> setEndOffsetsAsync(
    */
   ListenableFuture<Map<String, Object>> getMovingAveragesAsync(String id);
 
+  /**
+   * Get streamer metrics for a task. Retries on failure.
+   *
+   * Task-side is {@link SeekableStreamIndexTaskRunner#getRowStats}.
+   *
+   * @param id task id
+   */
+  ListenableFuture<Map<String, Object>> getStreamerMetrics(String id);

Review Comment:
   I don't think "streamer" is a term we use in Druid in the context of 
real-time ingestion.
   Could you please rename this method to align better with the existing 
terminology?
   Say `getAutoScalerMetrics()` or `getStreamConsumerMetrics()`.



##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.autoscaler;
+
+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.CostBasedAutoScaler;
+import 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScalerConfig;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.emitter.core.EventMap;
+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.indexing.MoreResources;
+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.concurrent.Executors;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler.OPTIMAL_TASK_COUNT_METRIC;
+
+/**
+ * Integration test for {@link CostBasedAutoScaler}.
+ * <p>
+ * Tests the autoscaler's ability to compute optimal task counts based on 
partition count and cost metrics (lag and idle time).
+ */
+public class CostBasedAutoScalerIntegrationTest 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 EmbeddedBroker broker = new EmbeddedBroker();
+  private final EmbeddedIndexer indexer = new EmbeddedIndexer();
+  private final EmbeddedOverlord overlord = new EmbeddedOverlord();
+  private final EmbeddedHistorical historical = new EmbeddedHistorical();
+  private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator();
+  private KafkaResource kafkaServer;
+
+  @Override
+  public EmbeddedDruidCluster createCluster()
+  {
+    final EmbeddedDruidCluster cluster = 
EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper();
+
+    kafkaServer = new KafkaResource()
+    {
+      @Override
+      public void start()
+      {
+        super.start();
+        createTopicWithPartitions(TOPIC, 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.useIncrementalCache", 
"ifSynced")
+            .addProperty("druid.manager.segments.pollDuration", "PT0.1s");
+
+    coordinator.addProperty("druid.manager.segments.useIncrementalCache", 
"ifSynced");
+
+    cluster.useLatchableEmitter()
+           .addServer(coordinator)
+           .addServer(overlord)
+           .addServer(indexer)
+           .addServer(broker)
+           .addServer(historical)
+           .addExtension(KafkaIndexTaskModule.class)
+           .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.5s")
+           .addResource(kafkaServer)
+           .addServer(new EmbeddedRouter());
+
+    return cluster;
+  }
+
+  @Test
+  @Timeout(45)
+  public void test_autoScaler_computesOptimalTaskCountAndProduceScaleDown()
+  {
+    final String superId = dataSource + "_super";
+    final int initialTaskCount = 10;
+
+    final CostBasedAutoScalerConfig autoScalerConfig = 
CostBasedAutoScalerConfig
+        .builder()
+        .enableTaskAutoScaler(true)
+        .taskCountMin(1)
+        .taskCountMax(100)
+        .taskCountStart(initialTaskCount)
+        .scaleActionPeriodMillis(1500)
+        .minTriggerScaleActionFrequencyMillis(3000)
+        // Weight configuration: strongly favor lag reduction over idle time
+        .lagWeight(0.9)
+        .idleWeight(0.1)
+        .build();
+
+    final KafkaSupervisorSpec spec = 
createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig, 
initialTaskCount);
+
+    // Submit the supervisor
+    Assertions.assertEquals(superId, cluster.callApi().postSupervisor(spec));
+
+    // Wait for the supervisor to be healthy and running
+    overlord.latchableEmitter().waitForEvent(event -> 
event.hasMetricName("task/run/time"));
+
+    // Wait for autoscaler to emit optimalTaskCount metric indicating 
scale-down
+    // We expect the optimal task count to 4
+    overlord.latchableEmitter().waitForEvent(
+        event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC)
+                      .hasValueMatching(Matchers.equalTo(6L))
+    );
+
+    // Suspend the supervisor
+    cluster.callApi().postSupervisor(spec.createSuspendedSpec());
+  }
+
+  @Test
+  public void test_autoScaler_computesOptimalTaskCountAndProducesScaleUp()
+  {
+    final String superId = dataSource + "_super_scaleup";
+
+    // Start with a low task count (1 task for 50 partitions) and produce a 
large amount of data
+    // to create lag pressure and low idle ratio, which should trigger a 
scale-up decision.
+    // With the ideal idle range [0.2, 0.6], a single overloaded task will 
have idle < 0.2,
+    // triggering the cost function to recommend more tasks.
+    final int lowInitialTaskCount = 1;
+
+    // Produce additional records to create a backlog / lag
+    // This ensures tasks are busy processing (low idle ratio)
+    Executors.newSingleThreadExecutor().submit(() -> 
produceRecordsToKafka(500_000, 20));
+
+    // These values were carefully handpicked to allow that test to pass in a 
stable manner.
+    final CostBasedAutoScalerConfig autoScalerConfig = 
CostBasedAutoScalerConfig
+        .builder()
+        .enableTaskAutoScaler(true)
+        .taskCountMin(1)
+        .taskCountMax(50)
+        .taskCountStart(lowInitialTaskCount)
+        .scaleActionPeriodMillis(500)
+        .minTriggerScaleActionFrequencyMillis(1000)
+        // Weight configuration: favor lag as the primary signal for scale-up 
scenarios
+        // High lag with low idle (overloaded) will trigger scale-up
+        .lagWeight(0.2)
+        .idleWeight(0.8)
+        .build();
+
+    final KafkaSupervisorSpec kafkaSupervisorSpec = 
createKafkaSupervisorWithAutoScaler(
+        superId,
+        autoScalerConfig,
+        lowInitialTaskCount
+    );
+
+    // Submit the supervisor
+    Assertions.assertEquals(superId, 
cluster.callApi().postSupervisor(kafkaSupervisorSpec));
+
+    // Wait for the supervisor to be healthy and running
+    overlord.latchableEmitter().waitForEvent(event -> 
event.hasMetricName("task/run/time"));

Review Comment:
   ```suggestion
       overlord.latchableEmitter().waitForEvent(event -> 
event.hasMetricName("task/run/time").hasDimension(DruidMetrics.DATASOURCE, 
dataSource));
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;

Review Comment:
   Nit: rename to better explain the usage
   ```suggestion
     private static final int MAX_INCREASE_IN_PARTITIONS_PER_TASK = 2;
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;
+  private static final int SCALE_DOWN_FACTOR_DISCRETE_DISTANCE = 
SCALE_UP_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;
+  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.costFunction = new WeightedCostFunction();
+
+    this.autoscalerExecutor = 
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));

Review Comment:
   ```suggestion
       this.autoscalerExecutor = 
Execs.scheduledSingleThreaded("CostBasedAutoScaler-" + 
StringUtils.encodeForFormat(spec.getId()));
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;
+  private static final int SCALE_DOWN_FACTOR_DISCRETE_DISTANCE = 
SCALE_UP_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;
+  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.costFunction = new WeightedCostFunction();
+
+    this.autoscalerExecutor = 
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+    this.metricBuilder = ServiceMetricEvent.builder()
+                                           
.setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId)
+                                           .setDimension(
+                                               DruidMetrics.STREAM,
+                                               
this.supervisor.getIoConfig().getStream()
+                                           );
+  }
+
+  @Override
+  public void start()
+  {
+    Callable<Integer> scaleAction = () -> 
computeOptimalTaskCount(this.collectMetrics());
+    Runnable onSuccessfulScale = () -> {
+    };
+
+    autoscalerExecutor.scheduleAtFixedRate(
+        supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, 
emitter),
+        config.getScaleActionPeriodMillis(),
+        config.getScaleActionPeriodMillis(),
+        TimeUnit.MILLISECONDS
+    );
+
+    log.info(
+        "CostBasedAutoScaler started for supervisorId [%s]: evaluating scaling 
every [%d]ms",

Review Comment:
   Super Nit: Druid-style logs generally avoid the space between the field name 
and its value.
   ```suggestion
           "CostBasedAutoScaler started for supervisorId[%s]: evaluating 
scaling every [%d]ms",
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/WeightedCostFunction.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.java.util.common.logger.Logger;
+
+/**
+ * Weighted cost function combining lag, idle time, and change distance 
metrics.
+ * Uses adaptive bounds for normalization based on recent history.
+ */
+public class WeightedCostFunction

Review Comment:
   I like the idea of the WeightedCostFunction, but I think we need to make it 
much more intuitive.
   
   #### Define requirements (already aligned with the current state of this PR):
   
   1. A function that computes cost.
   2. A task count with lower cost is better.
   3. `cost = lagCost * lagWeight + idlenessCost * idleWeight`
   4. Lower the task count, higher the predicted lag, higher the lag cost.
   5. Higher the task count, higher the predicted idleness, higher the idleness 
cost.
   
   #### Simplify computations
   
   1. Use linear scaling only, logarithmic scaling makes the terms difficult to 
reason about and debug.
   The diminishing returns effect is already enforced by the window (discrete 
distance). If more terms are needed to account for say, task operational 
overhead, we will add them in the future.
   2. Use only one mode i.e. do not invert scaling, even when lag is abnormally 
high.
   3. Use actual metrics instead of normalized or adaptive bounds.
   If a supervisor once saw a lag of 100M, the adaptive ratio would make a lag 
of 1M seem very small (`normalizedLag = 0.01` i.e. 1%). But in reality, a lag 
of 1M is bad too and needs to be given appropriate weight.
   4. Always perform cost computation even if idleness is in the accepted range 
(0.2-0.6 in the PR). This would help us validate the correctness of the formula 
against real clusters by verifying that the current task count gives minimal 
cost.
   
   We may re-introduce some of these enhancements in later patches once we have 
more data points using this auto-scaler, but it is best to start as simple as 
possible.
   
   #### Use intuitive metric e.g. compute time
   
   Connect the computed cost to an actual metric to make it more intuitive. The 
best metric I can think of is compute time or compute cycles, as it may be 
related to actual monetary cost of running tasks.
   
   For example, what if we could model the cost as follows:
   1. `lagCost = expected time (in seconds) required to recover current lag`
   2. `idlenessCost = total compute time (in seconds) wasted being idle in a 
single taskDuration`
   3. Intuitively, we can tell as task count increases, `lagCost` would 
increase and `idlenessCost` would decrease.
   
   The formula for these costs may be something like:
   ```
   lagCost
   = expected time (in seconds) required to recover current lag
   = currentAggregateLag / (proposedTaskCount * avgRateOfProcessing)
   
   where,
   currentAggregateLag = sum of current lag (in records) across all partitions
   avgRateOfProcessing = average of task moving averages
   ```
   
   ```
   idlenessCost
   = total time (in seconds) wasted being idle in a single taskDuration
   = total task run time * predicted idleness ratio
   
   where,
   total task run time = (proposedTaskCount * taskDuration)
   
   predicted idleness ratio = (proposedTaskCount / currentTaskCount) - (1 - 
avgPollToIdleRatio)
   
   e.g. if current poll-to-idle-ratio is 0.7, tasks are idle 70% of the time, 
so reducing task count by 70% will make tasks busy all the time (idleness ratio 
= 0).
   ```
   
   ##### Assumptions
   
   - Tasks are already at their peak processing rate and will remain at this 
rate.
   - poll-to-idle ratio scales linearly with task count. We may use some 
reasonable clamps for min (say 0.05) and max (say 0.95).
   
   
   



##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.autoscaler;
+
+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.CostBasedAutoScaler;
+import 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScalerConfig;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.emitter.core.EventMap;
+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.indexing.MoreResources;
+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.concurrent.Executors;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler.OPTIMAL_TASK_COUNT_METRIC;
+
+/**
+ * Integration test for {@link CostBasedAutoScaler}.
+ * <p>
+ * Tests the autoscaler's ability to compute optimal task counts based on 
partition count and cost metrics (lag and idle time).
+ */
+public class CostBasedAutoScalerIntegrationTest 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 EmbeddedBroker broker = new EmbeddedBroker();
+  private final EmbeddedIndexer indexer = new EmbeddedIndexer();
+  private final EmbeddedOverlord overlord = new EmbeddedOverlord();
+  private final EmbeddedHistorical historical = new EmbeddedHistorical();
+  private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator();
+  private KafkaResource kafkaServer;
+
+  @Override
+  public EmbeddedDruidCluster createCluster()
+  {
+    final EmbeddedDruidCluster cluster = 
EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper();
+
+    kafkaServer = new KafkaResource()
+    {
+      @Override
+      public void start()
+      {
+        super.start();
+        createTopicWithPartitions(TOPIC, 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.useIncrementalCache", 
"ifSynced")
+            .addProperty("druid.manager.segments.pollDuration", "PT0.1s");
+
+    coordinator.addProperty("druid.manager.segments.useIncrementalCache", 
"ifSynced");
+
+    cluster.useLatchableEmitter()
+           .addServer(coordinator)
+           .addServer(overlord)
+           .addServer(indexer)
+           .addServer(broker)
+           .addServer(historical)
+           .addExtension(KafkaIndexTaskModule.class)
+           .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.5s")
+           .addResource(kafkaServer)
+           .addServer(new EmbeddedRouter());
+
+    return cluster;
+  }
+
+  @Test
+  @Timeout(45)
+  public void test_autoScaler_computesOptimalTaskCountAndProduceScaleDown()
+  {
+    final String superId = dataSource + "_super";
+    final int initialTaskCount = 10;
+
+    final CostBasedAutoScalerConfig autoScalerConfig = 
CostBasedAutoScalerConfig
+        .builder()
+        .enableTaskAutoScaler(true)
+        .taskCountMin(1)
+        .taskCountMax(100)
+        .taskCountStart(initialTaskCount)
+        .scaleActionPeriodMillis(1500)
+        .minTriggerScaleActionFrequencyMillis(3000)
+        // Weight configuration: strongly favor lag reduction over idle time
+        .lagWeight(0.9)
+        .idleWeight(0.1)
+        .build();
+
+    final KafkaSupervisorSpec spec = 
createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig, 
initialTaskCount);
+
+    // Submit the supervisor
+    Assertions.assertEquals(superId, cluster.callApi().postSupervisor(spec));
+
+    // Wait for the supervisor to be healthy and running
+    overlord.latchableEmitter().waitForEvent(event -> 
event.hasMetricName("task/run/time"));

Review Comment:
   Wait for the metric from the same datasource.
   
   ```suggestion
       overlord.latchableEmitter().waitForEvent(event -> 
event.hasMetricName("task/run/time").hasDimension(DruidMetrics.DATASOURCE, 
dataSource));
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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.

Review Comment:
   ```suggestion
    * Scale-up and scale-down are both performed proactively.
    * Future versions may perform scale-down on task rollover only.
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;
+  private static final int SCALE_DOWN_FACTOR_DISCRETE_DISTANCE = 
SCALE_UP_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;
+  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.costFunction = new WeightedCostFunction();
+
+    this.autoscalerExecutor = 
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+    this.metricBuilder = ServiceMetricEvent.builder()
+                                           
.setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId)
+                                           .setDimension(
+                                               DruidMetrics.STREAM,
+                                               
this.supervisor.getIoConfig().getStream()
+                                           );
+  }
+
+  @Override
+  public void start()
+  {
+    Callable<Integer> scaleAction = () -> 
computeOptimalTaskCount(this.collectMetrics());
+    Runnable onSuccessfulScale = () -> {
+    };
+
+    autoscalerExecutor.scheduleAtFixedRate(
+        supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, 
emitter),
+        config.getScaleActionPeriodMillis(),
+        config.getScaleActionPeriodMillis(),
+        TimeUnit.MILLISECONDS
+    );
+
+    log.info(
+        "CostBasedAutoScaler started for supervisorId [%s]: evaluating scaling 
every [%d]ms",
+        supervisorId,
+        config.getScaleActionPeriodMillis()
+    );
+  }
+
+  @Override
+  public void stop()
+  {
+    autoscalerExecutor.shutdownNow();
+    log.info("CostBasedAutoScaler stopped for supervisorId [%s]", 
supervisorId);
+  }
+
+  @Override
+  public void reset()
+  {
+    // No-op.
+  }
+
+  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 double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+    return new CostMetrics(lagStats.getAvgLag(), currentTaskCount, 
partitionCount, pollIdleRatio);
+  }
+
+  /**
+   * 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(CostMetrics metrics)
+  {
+    if (metrics == null) {
+      log.debug("No metrics available yet for supervisorId [%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.computeValidTaskCounts(partitionCount, currentTaskCount);
+
+    if (validTaskCounts.length == 0) {
+      log.warn("No valid task counts after applying constraints for 
supervisorId [%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 supervisorId [%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;
+
+    for (int taskCount : validTaskCounts) {
+      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.debug(
+        "Cost-based scaling evaluation for supervisorId [%s]: current=%d, 
optimal=%d, cost=%.4f, "
+        + "avgPartitionLag=%.2f, pollIdleRatio=%.3f",
+        supervisorId,
+        metrics.getCurrentTaskCount(),
+        optimalTaskCount,
+        optimalCost,
+        metrics.getAvgPartitionLag(),
+        metrics.getPollIdleRatio()
+    );
+
+    if (optimalTaskCount == currentTaskCount) {
+      return -1;
+    }
+    // Temporarily, we equalize scaleup and scaledown effects, due to 
uncompleted state of task rollover.
+    // The behaviour will be changed by complementing task rollover state 
machine.
+    return optimalTaskCount;
+  }
+
+  /**
+   * Generates valid task counts based on partitions-per-task ratios.
+   * This enables gradual scaling and avoids large jumps.
+   * Limits the range of task counts considered to avoid excessive computation.
+   *
+   * @return sorted list of valid task counts within bounds
+   */
+  static int[] computeValidTaskCounts(int partitionCount, int currentTaskCount)
+  {
+    if (partitionCount <= 0) {
+      return new int[]{};
+    }
+
+    List<Integer> result = new ArrayList<>();
+    final int currentPartitionsPerTask = partitionCount / currentTaskCount;
+    // To avoid confusion: minimum partitions per task means maximum amount of 
tasks (scale up) and vice versa.

Review Comment:
   ```suggestion
       // Minimum partitions per task corresponds to maximum number of tasks 
(scale up) and vice versa.
   ```



##########
server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java:
##########
@@ -66,7 +66,7 @@ default ListenableFuture<Void> stopAsync()
 
   SupervisorStateManager.State getState();
 
-  default Map<String, Map<String, Object>> getStats()
+  default Map<String, Map<String, Object>> getStats(boolean 
includeOnlyStreamerStats)

Review Comment:
   Please rename this argument to align better with Druid terminology.
   Please also add a short javadoc to this method.



##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.autoscaler;
+
+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.CostBasedAutoScaler;
+import 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScalerConfig;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.emitter.core.EventMap;
+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.indexing.MoreResources;
+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.concurrent.Executors;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler.OPTIMAL_TASK_COUNT_METRIC;
+
+/**
+ * Integration test for {@link CostBasedAutoScaler}.
+ * <p>
+ * Tests the autoscaler's ability to compute optimal task counts based on 
partition count and cost metrics (lag and idle time).
+ */
+public class CostBasedAutoScalerIntegrationTest 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 EmbeddedBroker broker = new EmbeddedBroker();
+  private final EmbeddedIndexer indexer = new EmbeddedIndexer();
+  private final EmbeddedOverlord overlord = new EmbeddedOverlord();
+  private final EmbeddedHistorical historical = new EmbeddedHistorical();
+  private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator();
+  private KafkaResource kafkaServer;
+
+  @Override
+  public EmbeddedDruidCluster createCluster()
+  {
+    final EmbeddedDruidCluster cluster = 
EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper();
+
+    kafkaServer = new KafkaResource()
+    {
+      @Override
+      public void start()
+      {
+        super.start();
+        createTopicWithPartitions(TOPIC, 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.useIncrementalCache", 
"ifSynced")
+            .addProperty("druid.manager.segments.pollDuration", "PT0.1s");
+
+    coordinator.addProperty("druid.manager.segments.useIncrementalCache", 
"ifSynced");
+
+    cluster.useLatchableEmitter()
+           .addServer(coordinator)
+           .addServer(overlord)
+           .addServer(indexer)
+           .addServer(broker)
+           .addServer(historical)
+           .addExtension(KafkaIndexTaskModule.class)
+           .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.5s")
+           .addResource(kafkaServer)
+           .addServer(new EmbeddedRouter());
+
+    return cluster;
+  }
+
+  @Test
+  @Timeout(45)
+  public void test_autoScaler_computesOptimalTaskCountAndProduceScaleDown()
+  {
+    final String superId = dataSource + "_super";
+    final int initialTaskCount = 10;
+
+    final CostBasedAutoScalerConfig autoScalerConfig = 
CostBasedAutoScalerConfig
+        .builder()
+        .enableTaskAutoScaler(true)
+        .taskCountMin(1)
+        .taskCountMax(100)
+        .taskCountStart(initialTaskCount)
+        .scaleActionPeriodMillis(1500)
+        .minTriggerScaleActionFrequencyMillis(3000)
+        // Weight configuration: strongly favor lag reduction over idle time
+        .lagWeight(0.9)
+        .idleWeight(0.1)
+        .build();
+
+    final KafkaSupervisorSpec spec = 
createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig, 
initialTaskCount);
+
+    // Submit the supervisor
+    Assertions.assertEquals(superId, cluster.callApi().postSupervisor(spec));
+
+    // Wait for the supervisor to be healthy and running
+    overlord.latchableEmitter().waitForEvent(event -> 
event.hasMetricName("task/run/time"));
+
+    // Wait for autoscaler to emit optimalTaskCount metric indicating 
scale-down
+    // We expect the optimal task count to 4
+    overlord.latchableEmitter().waitForEvent(
+        event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC)
+                      .hasValueMatching(Matchers.equalTo(6L))
+    );
+
+    // Suspend the supervisor
+    cluster.callApi().postSupervisor(spec.createSuspendedSpec());
+  }
+
+  @Test
+  public void test_autoScaler_computesOptimalTaskCountAndProducesScaleUp()
+  {
+    final String superId = dataSource + "_super_scaleup";
+
+    // Start with a low task count (1 task for 50 partitions) and produce a 
large amount of data
+    // to create lag pressure and low idle ratio, which should trigger a 
scale-up decision.
+    // With the ideal idle range [0.2, 0.6], a single overloaded task will 
have idle < 0.2,
+    // triggering the cost function to recommend more tasks.
+    final int lowInitialTaskCount = 1;
+
+    // Produce additional records to create a backlog / lag
+    // This ensures tasks are busy processing (low idle ratio)
+    Executors.newSingleThreadExecutor().submit(() -> 
produceRecordsToKafka(500_000, 20));
+
+    // These values were carefully handpicked to allow that test to pass in a 
stable manner.
+    final CostBasedAutoScalerConfig autoScalerConfig = 
CostBasedAutoScalerConfig
+        .builder()
+        .enableTaskAutoScaler(true)
+        .taskCountMin(1)
+        .taskCountMax(50)
+        .taskCountStart(lowInitialTaskCount)
+        .scaleActionPeriodMillis(500)
+        .minTriggerScaleActionFrequencyMillis(1000)
+        // Weight configuration: favor lag as the primary signal for scale-up 
scenarios
+        // High lag with low idle (overloaded) will trigger scale-up
+        .lagWeight(0.2)
+        .idleWeight(0.8)
+        .build();
+
+    final KafkaSupervisorSpec kafkaSupervisorSpec = 
createKafkaSupervisorWithAutoScaler(
+        superId,
+        autoScalerConfig,
+        lowInitialTaskCount
+    );
+
+    // Submit the supervisor
+    Assertions.assertEquals(superId, 
cluster.callApi().postSupervisor(kafkaSupervisorSpec));
+
+    // Wait for the supervisor to be healthy and running
+    overlord.latchableEmitter().waitForEvent(event -> 
event.hasMetricName("task/run/time"));
+
+    // First, wait for any optimalTaskCount metric to verify the autoscaler is 
running
+    overlord.latchableEmitter().waitForEvent(
+        event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC)

Review Comment:
   ```suggestion
           event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC)
                         .hasDimension(DruidMetrics.SUPERVISOR_ID, superId)
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java:
##########
@@ -4381,6 +4374,39 @@ 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(true);
+    if (taskMetrics.isEmpty()) {
+      return 1.;

Review Comment:
   Returning 1 (full idle) here would cause the auto-scaler to think that the 
tasks are doing nothing and cause a scale-down, when in fact the tasks failed 
to return the metrics and may be in a bad state. Scaling down might further 
worsen the problem.
   
   Should we return 0 instead?



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;
+  private static final int SCALE_DOWN_FACTOR_DISCRETE_DISTANCE = 
SCALE_UP_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;
+  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.costFunction = new WeightedCostFunction();
+
+    this.autoscalerExecutor = 
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+    this.metricBuilder = ServiceMetricEvent.builder()
+                                           
.setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId)
+                                           .setDimension(
+                                               DruidMetrics.STREAM,
+                                               
this.supervisor.getIoConfig().getStream()
+                                           );
+  }
+
+  @Override
+  public void start()
+  {
+    Callable<Integer> scaleAction = () -> 
computeOptimalTaskCount(this.collectMetrics());
+    Runnable onSuccessfulScale = () -> {
+    };
+
+    autoscalerExecutor.scheduleAtFixedRate(
+        supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, 
emitter),
+        config.getScaleActionPeriodMillis(),
+        config.getScaleActionPeriodMillis(),
+        TimeUnit.MILLISECONDS
+    );
+
+    log.info(
+        "CostBasedAutoScaler started for supervisorId [%s]: evaluating scaling 
every [%d]ms",
+        supervisorId,
+        config.getScaleActionPeriodMillis()
+    );
+  }
+
+  @Override
+  public void stop()
+  {
+    autoscalerExecutor.shutdownNow();
+    log.info("CostBasedAutoScaler stopped for supervisorId [%s]", 
supervisorId);
+  }
+
+  @Override
+  public void reset()
+  {
+    // No-op.
+  }
+
+  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 double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+    return new CostMetrics(lagStats.getAvgLag(), currentTaskCount, 
partitionCount, pollIdleRatio);
+  }
+
+  /**
+   * 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(CostMetrics metrics)
+  {
+    if (metrics == null) {
+      log.debug("No metrics available yet for supervisorId [%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.computeValidTaskCounts(partitionCount, currentTaskCount);
+
+    if (validTaskCounts.length == 0) {
+      log.warn("No valid task counts after applying constraints for 
supervisorId [%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 supervisorId [%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;
+
+    for (int taskCount : validTaskCounts) {
+      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.debug(
+        "Cost-based scaling evaluation for supervisorId [%s]: current=%d, 
optimal=%d, cost=%.4f, "
+        + "avgPartitionLag=%.2f, pollIdleRatio=%.3f",
+        supervisorId,
+        metrics.getCurrentTaskCount(),
+        optimalTaskCount,
+        optimalCost,
+        metrics.getAvgPartitionLag(),
+        metrics.getPollIdleRatio()
+    );
+
+    if (optimalTaskCount == currentTaskCount) {
+      return -1;
+    }
+    // Temporarily, we equalize scaleup and scaledown effects, due to 
uncompleted state of task rollover.
+    // The behaviour will be changed by complementing task rollover state 
machine.

Review Comment:
   ```suggestion
       // Perform both scale-up and scale-down proactively
       // Future versions may perform scale-down on task rollover only
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;
+  private static final int SCALE_DOWN_FACTOR_DISCRETE_DISTANCE = 
SCALE_UP_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;
+  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.costFunction = new WeightedCostFunction();
+
+    this.autoscalerExecutor = 
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+    this.metricBuilder = ServiceMetricEvent.builder()
+                                           
.setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId)
+                                           .setDimension(
+                                               DruidMetrics.STREAM,
+                                               
this.supervisor.getIoConfig().getStream()
+                                           );
+  }
+
+  @Override
+  public void start()
+  {
+    Callable<Integer> scaleAction = () -> 
computeOptimalTaskCount(this.collectMetrics());
+    Runnable onSuccessfulScale = () -> {
+    };
+
+    autoscalerExecutor.scheduleAtFixedRate(
+        supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, 
emitter),
+        config.getScaleActionPeriodMillis(),
+        config.getScaleActionPeriodMillis(),
+        TimeUnit.MILLISECONDS
+    );
+
+    log.info(
+        "CostBasedAutoScaler started for supervisorId [%s]: evaluating scaling 
every [%d]ms",
+        supervisorId,
+        config.getScaleActionPeriodMillis()
+    );
+  }
+
+  @Override
+  public void stop()
+  {
+    autoscalerExecutor.shutdownNow();
+    log.info("CostBasedAutoScaler stopped for supervisorId [%s]", 
supervisorId);
+  }
+
+  @Override
+  public void reset()
+  {
+    // No-op.
+  }
+
+  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 double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+    return new CostMetrics(lagStats.getAvgLag(), currentTaskCount, 
partitionCount, pollIdleRatio);
+  }
+
+  /**
+   * 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(CostMetrics metrics)
+  {
+    if (metrics == null) {
+      log.debug("No metrics available yet for supervisorId [%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.computeValidTaskCounts(partitionCount, currentTaskCount);
+
+    if (validTaskCounts.length == 0) {
+      log.warn("No valid task counts after applying constraints for 
supervisorId [%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 supervisorId [%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;
+
+    for (int taskCount : validTaskCounts) {
+      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.debug(
+        "Cost-based scaling evaluation for supervisorId [%s]: current=%d, 
optimal=%d, cost=%.4f, "
+        + "avgPartitionLag=%.2f, pollIdleRatio=%.3f",
+        supervisorId,
+        metrics.getCurrentTaskCount(),
+        optimalTaskCount,
+        optimalCost,
+        metrics.getAvgPartitionLag(),
+        metrics.getPollIdleRatio()
+    );
+
+    if (optimalTaskCount == currentTaskCount) {
+      return -1;
+    }
+    // Temporarily, we equalize scaleup and scaledown effects, due to 
uncompleted state of task rollover.
+    // The behaviour will be changed by complementing task rollover state 
machine.
+    return optimalTaskCount;
+  }
+
+  /**
+   * Generates valid task counts based on partitions-per-task ratios.
+   * This enables gradual scaling and avoids large jumps.
+   * Limits the range of task counts considered to avoid excessive computation.
+   *
+   * @return sorted list of valid task counts within bounds
+   */
+  static int[] computeValidTaskCounts(int partitionCount, int currentTaskCount)
+  {
+    if (partitionCount <= 0) {
+      return new int[]{};
+    }
+
+    List<Integer> result = new ArrayList<>();
+    final int currentPartitionsPerTask = partitionCount / currentTaskCount;
+    // To avoid confusion: minimum partitions per task means maximum amount of 
tasks (scale up) and vice versa.
+    final int minPartitionsPerTask = Math.max(1, currentPartitionsPerTask - 
SCALE_UP_FACTOR_DISCRETE_DISTANCE);
+    final int maxPartitionsPerTask = Math.min(
+        partitionCount,
+        Math.max(
+            minPartitionsPerTask,
+            currentPartitionsPerTask + SCALE_DOWN_FACTOR_DISCRETE_DISTANCE
+        )
+    );
+
+    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);
+      }

Review Comment:
   Maybe just use a set to simplify this computation.



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;
+  private static final int SCALE_DOWN_FACTOR_DISCRETE_DISTANCE = 
SCALE_UP_FACTOR_DISCRETE_DISTANCE * 2;

Review Comment:
   ```suggestion
     private static final int MAX_DECREASE_IN_PARTITIONS_PER_TASK = 
SCALE_UP_FACTOR_DISCRETE_DISTANCE * 2;
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * 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_UP_FACTOR_DISCRETE_DISTANCE = 2;
+  private static final int SCALE_DOWN_FACTOR_DISCRETE_DISTANCE = 
SCALE_UP_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;
+  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.costFunction = new WeightedCostFunction();
+
+    this.autoscalerExecutor = 
Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId()));
+    this.metricBuilder = ServiceMetricEvent.builder()
+                                           
.setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId)
+                                           .setDimension(
+                                               DruidMetrics.STREAM,
+                                               
this.supervisor.getIoConfig().getStream()
+                                           );
+  }
+
+  @Override
+  public void start()
+  {
+    Callable<Integer> scaleAction = () -> 
computeOptimalTaskCount(this.collectMetrics());
+    Runnable onSuccessfulScale = () -> {
+    };
+
+    autoscalerExecutor.scheduleAtFixedRate(
+        supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, 
emitter),
+        config.getScaleActionPeriodMillis(),
+        config.getScaleActionPeriodMillis(),
+        TimeUnit.MILLISECONDS
+    );
+
+    log.info(
+        "CostBasedAutoScaler started for supervisorId [%s]: evaluating scaling 
every [%d]ms",
+        supervisorId,
+        config.getScaleActionPeriodMillis()
+    );
+  }
+
+  @Override
+  public void stop()
+  {
+    autoscalerExecutor.shutdownNow();
+    log.info("CostBasedAutoScaler stopped for supervisorId [%s]", 
supervisorId);
+  }
+
+  @Override
+  public void reset()
+  {
+    // No-op.
+  }
+
+  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 double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+    return new CostMetrics(lagStats.getAvgLag(), currentTaskCount, 
partitionCount, pollIdleRatio);
+  }
+
+  /**
+   * 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(CostMetrics metrics)
+  {
+    if (metrics == null) {
+      log.debug("No metrics available yet for supervisorId [%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.computeValidTaskCounts(partitionCount, currentTaskCount);
+
+    if (validTaskCounts.length == 0) {
+      log.warn("No valid task counts after applying constraints for 
supervisorId [%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 supervisorId [%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;
+
+    for (int taskCount : validTaskCounts) {
+      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.debug(
+        "Cost-based scaling evaluation for supervisorId [%s]: current=%d, 
optimal=%d, cost=%.4f, "
+        + "avgPartitionLag=%.2f, pollIdleRatio=%.3f",
+        supervisorId,
+        metrics.getCurrentTaskCount(),
+        optimalTaskCount,
+        optimalCost,
+        metrics.getAvgPartitionLag(),
+        metrics.getPollIdleRatio()
+    );
+
+    if (optimalTaskCount == currentTaskCount) {
+      return -1;
+    }
+    // Temporarily, we equalize scaleup and scaledown effects, due to 
uncompleted state of task rollover.
+    // The behaviour will be changed by complementing task rollover state 
machine.
+    return optimalTaskCount;
+  }
+
+  /**
+   * Generates valid task counts based on partitions-per-task ratios.
+   * This enables gradual scaling and avoids large jumps.
+   * Limits the range of task counts considered to avoid excessive computation.
+   *
+   * @return sorted list of valid task counts within bounds
+   */
+  static int[] computeValidTaskCounts(int partitionCount, int currentTaskCount)
+  {
+    if (partitionCount <= 0) {
+      return new int[]{};
+    }
+
+    List<Integer> result = new ArrayList<>();
+    final int currentPartitionsPerTask = partitionCount / currentTaskCount;
+    // To avoid confusion: minimum partitions per task means maximum amount of 
tasks (scale up) and vice versa.
+    final int minPartitionsPerTask = Math.max(1, currentPartitionsPerTask - 
SCALE_UP_FACTOR_DISCRETE_DISTANCE);
+    final int maxPartitionsPerTask = Math.min(
+        partitionCount,
+        Math.max(
+            minPartitionsPerTask,
+            currentPartitionsPerTask + SCALE_DOWN_FACTOR_DISCRETE_DISTANCE
+        )
+    );
+
+    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);
+      }
+    }
+    return result.stream().mapToInt(Integer::intValue).toArray();

Review Comment:
   Nit: Is conversion to array still needed? Can we just return List or Set 
instead?



-- 
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]

Reply via email to