Fly-Style commented on code in PR #18819:
URL: https://github.com/apache/druid/pull/18819#discussion_r2634904558


##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,360 @@
+/*
+ * 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.common.stats.DropwizardRowIngestionMeters;
+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.SeekableStreamIndexTaskRunner;
+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 org.apache.druid.segment.incremental.RowIngestionMeters;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+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 weighted cost function combining lag recovery time (seconds) and 
idleness cost (seconds)
+ * to determine optimal task counts.
+ * <p>
+ * Candidate task counts are derived by scanning a bounded window of 
partitions-per-task (PPT) values
+ * around the current PPT, then converting those to task counts. This allows 
non-divisor task counts
+ * while keeping changes gradual (no large jumps).
+ * <p>
+ * Scale-up and scale-down are both evaluated proactively.
+ * Future versions may perform scale-down on task rollover only.
+ */
+public class CostBasedAutoScaler implements SupervisorTaskAutoScaler
+{
+  private static final EmittingLogger log = new 
EmittingLogger(CostBasedAutoScaler.class);
+
+  private static final int MAX_INCREASE_IN_PARTITIONS_PER_TASK = 2;
+  private static final int MAX_DECREASE_IN_PARTITIONS_PER_TASK = 
MAX_INCREASE_IN_PARTITIONS_PER_TASK * 2;
+  public static final String AVG_LAG_METRIC = 
"task/autoScaler/costBased/avgLag";
+  public static final String AVG_IDLE_METRIC = 
"task/autoScaler/costBased/pollIdleAvg";
+  public static final String 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("CostBasedAutoScaler-" + 
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 Map<String, Map<String, Object>> taskStats = supervisor.getStats();
+    final double movingAvgRate = extractMovingAverage(taskStats, 
DropwizardRowIngestionMeters.ONE_MINUTE_NAME);
+    final double pollIdleRatio = extractPollIdleRatio(taskStats);
+
+    final double avgPartitionLag = lagStats.getAvgLag();
+
+    // Use an actual 15-minute moving average processing rate if available
+    final double avgProcessingRate;
+    if (movingAvgRate > 0) {
+      avgProcessingRate = movingAvgRate;
+    } else {
+      // Fallback: estimate processing rate based on idle ratio
+      final double utilizationRatio = Math.max(0.01, 1.0 - pollIdleRatio);
+      avgProcessingRate = config.getDefaultProcessingRate() * utilizationRatio;

Review Comment:
   The actual proble, that fallback is solving that it is required for the 
first (couple) iteration(s) when we don't have any data about some previous 
processing rate. You neither want to scale up or scale down drastically on this 
critical period. 
   Actually we may just return -1 in that case. Let me know WDYT about this.



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