kfaraz commented on code in PR #18819: URL: https://github.com/apache/druid/pull/18819#discussion_r2621792154
########## indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java: ########## @@ -0,0 +1,256 @@ +/* + * 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 and scale-down are both performed 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 MIN_INCREASE_IN_PARTITIONS_PER_TASK = MAX_INCREASE_IN_PARTITIONS_PER_TASK * 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("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(), Review Comment: Based on the feedback from @jtuglu1 , I think we might need to reconsider this. But rather than add a new config, I wonder if we can't improve this logic a bit. Options: 1. `startDelay = Math.min(taskDuration, 30 mins)`. So we consider scaling up only after the current tasks have run for a bit. 2. `startDelay = 3 * config.getScaleActionPeriodMillis()`. This could work too, and seems reasonable for most cases. 3. Add a separate config `scaleActionStartDelayMillis` (so that we remain aligned with the existing behaviour of lag-based auto-scaler), whose default value is `3 * config.getScaleActionPeriodMillis()`. @jtuglu1 , @Fly-Style , which one do you guys prefer the most? -- 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]
