github-advanced-security[bot] commented on code in PR #18819:
URL: https://github.com/apache/druid/pull/18819#discussion_r2594197748


##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.commons.collections4.queue.CircularFifoQueue;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import 
org.apache.druid.indexing.overlord.supervisor.autoscaler.AggregateFunction;
+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 javax.validation.constraints.NotNull;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * 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 predefined 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 double MESSAGE_GAP_FULL_IDLE_MILLIS = 5000.0;
+  private static final int SCALE_FACTOR_DISCRETE_DISTANCE = 2;
+
+  private final String dataSource;
+  private final CircularFifoQueue<CostMetrics> metricsQueue;
+  private final ScheduledExecutorService metricsCollectionExec;
+  private final ScheduledExecutorService scalingDecisionExec;
+  private final SupervisorSpec spec;
+  private final SeekableStreamSupervisor supervisor;
+  private final CostBasedAutoScalerConfig config;
+  private final ServiceEmitter emitter;
+  private final ServiceMetricEvent.Builder metricBuilder;
+  private final WeightedCostFunction costFunction;
+
+  private final ReentrantLock lock = new ReentrantLock(true);
+
+  public CostBasedAutoScaler(
+      SeekableStreamSupervisor supervisor,
+      String dataSource,
+      CostBasedAutoScalerConfig config,
+      SupervisorSpec spec,
+      ServiceEmitter emitter
+  )
+  {
+    this.dataSource = dataSource;
+    this.config = config;
+    this.spec = spec;
+    this.supervisor = supervisor;
+    this.emitter = emitter;
+
+    final String supervisorId = StringUtils.format("Supervisor-%s", 
dataSource);
+    final int queueSize = (int) (config.getMetricsCollectionRangeMillis()
+                                 / 
config.getMetricsCollectionIntervalMillis()) + 1;
+
+    this.metricsQueue = new CircularFifoQueue<>(queueSize);
+    this.costFunction = new WeightedCostFunction();
+
+    this.metricsCollectionExec = Execs.scheduledSingleThreaded(
+        StringUtils.encodeForFormat(supervisorId) + "-CostBasedMetrics-%d"
+    );
+    this.scalingDecisionExec = Execs.scheduledSingleThreaded(
+        StringUtils.encodeForFormat(supervisorId) + "-CostBasedScaling-%d"
+    );
+
+    this.metricBuilder = ServiceMetricEvent.builder()
+                                           
.setDimension(DruidMetrics.DATASOURCE, dataSource)
+                                           .setDimension(
+                                               DruidMetrics.STREAM,
+                                               
this.supervisor.getIoConfig().getStream()
+                                           );
+  }
+
+  @Override
+  public void start()
+  {
+    Callable<Integer> scaleAction = () -> {
+      lock.lock();
+      try {
+        return computeOptimalTaskCount(new ArrayList<>(metricsQueue));
+      }
+      finally {
+        lock.unlock();
+      }
+    };
+
+    Runnable onSuccessfulScale = () -> {
+      lock.lock();
+      try {
+        metricsQueue.clear();
+      }
+      finally {
+        lock.unlock();
+      }
+    };
+
+    metricsCollectionExec.scheduleAtFixedRate(
+        collectMetrics(),
+        config.getScaleActionStartDelayMillis(),
+        config.getMetricsCollectionIntervalMillis(),
+        TimeUnit.MILLISECONDS
+    );
+
+    scalingDecisionExec.scheduleAtFixedRate(
+        supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, 
emitter),
+        config.getScaleActionStartDelayMillis() + 
config.getMetricsCollectionRangeMillis(),
+        config.getScaleActionPeriodMillis(),
+        TimeUnit.MILLISECONDS
+    );
+
+    log.info(
+        "CostBasedAutoScaler started for dataSource [%s]: collecting metrics 
every [%d]ms, "
+        + "evaluating scaling every [%d]ms, queue size [%d]",
+        dataSource,
+        config.getMetricsCollectionIntervalMillis(),
+        config.getScaleActionPeriodMillis(),
+        metricsQueue.maxSize()
+    );
+  }
+
+  @Override
+  public void stop()
+  {
+    scalingDecisionExec.shutdownNow();
+    metricsCollectionExec.shutdownNow();
+    log.info("CostBasedAutoScaler stopped for dataSource [%s]", dataSource);
+  }
+
+  @Override
+  public void reset()
+  {
+    lock.lock();
+    try {
+      metricsQueue.clear();
+      log.info("CostBasedAutoScaler reset for dataSource [%s]", dataSource);
+    }
+    catch (Exception e) {
+      log.warn(e, "Error while resetting CostBasedAutoScaler for dataSource 
[%s]", dataSource);
+    }
+    finally {
+      lock.unlock();
+    }
+  }
+
+  private Runnable collectMetrics()
+  {
+    return () -> {
+      if (spec.isSuspended()) {
+        log.debug("Supervisor [%s] is suspended, skipping metrics collection", 
dataSource);
+        return;
+      }
+
+      final LagStats lagStats = supervisor.computeLagStats();
+      if (lagStats == null) {
+        log.debug("Lag stats unavailable for dataSource [%s], skipping 
collection", dataSource);
+        return;
+      }
+
+      final int currentTaskCount = supervisor.getActiveTaskGroupsCount();
+      final int partitionCount = supervisor.getPartitionCount();
+      final double avgPartitionLag = partitionCount > 0
+                                     ? (double) lagStats.getTotalLag() / 
partitionCount
+                                     : 0.0;
+
+      final double pollIdleRatio = extractPollIdleRatio();
+
+      final CostMetrics metrics = new CostMetrics(
+          System.currentTimeMillis(),
+          avgPartitionLag,
+          currentTaskCount,
+          partitionCount,
+          pollIdleRatio
+      );
+
+      lock.lock();
+      try {
+        metricsQueue.offer(metrics);

Review Comment:
   ## Ignored error status of call
   
   Method run ignores exceptional return value of 
CircularFifoQueue<CostMetrics>.offer.
   
   [Show more 
details](https://github.com/apache/druid/security/code-scanning/10578)



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