kfaraz commented on code in PR #18819: URL: https://github.com/apache/druid/pull/18819#discussion_r2610714982
########## indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java: ########## @@ -0,0 +1,275 @@ +/* + * 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.LinkedHashMap; +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.atomic.AtomicReference; + +/** + * Cost-based auto-scaler for seekable stream supervisors. + * Uses a cost function combining lag and idle time metrics to determine optimal task counts. + * Task counts are selected from 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 int SCALE_FACTOR_DISCRETE_DISTANCE = 2; + public static final String OPTIMAL_TASK_COUNT_METRIC = "task/autoScaler/costBased/optimalTaskCount"; + + private static final Map<Integer, int[]> FACTORS_CACHE = new LinkedHashMap<>(); + private static final int FACTORS_CACHE_MAX_SIZE = 10; // Enough for most scenarios, almost capacity * loadFactor. + + private final String supervisorId; + private final SeekableStreamSupervisor supervisor; + private final ServiceEmitter emitter; + private final SupervisorSpec spec; + private final CostBasedAutoScalerConfig config; + private final ServiceMetricEvent.Builder metricBuilder; + /** + * Atomic reference to CostMetrics object. All operations must be performed + * with sequentially consistent semantics (volatile reads/writes). + * However, it may be fine-tuned with acquire/release semantics, + * but requires careful reasoning about correctness. + */ + private final AtomicReference<CostMetrics> currentMetrics; + private final ScheduledExecutorService autoscalerExecutor; + private final WeightedCostFunction costFunction; + + public CostBasedAutoScaler( + SeekableStreamSupervisor supervisor, + CostBasedAutoScalerConfig config, + SupervisorSpec spec, + ServiceEmitter emitter + ) + { + this.config = config; + this.spec = spec; + this.supervisor = supervisor; + this.supervisorId = spec.getId(); + this.emitter = emitter; + + this.currentMetrics = new AtomicReference<>(null); + this.costFunction = new WeightedCostFunction(); + + this.autoscalerExecutor = Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(spec.getId())); + this.metricBuilder = ServiceMetricEvent.builder() + .setDimension(DruidMetrics.DATASOURCE, supervisorId) + .setDimension( + DruidMetrics.STREAM, + this.supervisor.getIoConfig().getStream() + ); + } + + @Override + public void start() + { + Callable<Integer> scaleAction = () -> computeOptimalTaskCount(currentMetrics); + Runnable onSuccessfulScale = () -> currentMetrics.set(null); + + autoscalerExecutor.scheduleAtFixedRate( + this::collectMetrics, + config.getMetricsCollectionIntervalMillis(), + config.getMetricsCollectionIntervalMillis(), + TimeUnit.MILLISECONDS + ); + + autoscalerExecutor.scheduleAtFixedRate( + supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale, emitter), Review Comment: Instead of scheduling the collection of metrics and the trigger of scale action separately, why not do it in a single Runnable? It makes sense to collect metrics as frequently as we do the scale action computation. If we collect metric values more frequently, we are just going to overwrite them. If we collect them less frequently, the scaling action might end up being a no-op. -- 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]
