zhangyue19921010 commented on a change in pull request #10524: URL: https://github.com/apache/druid/pull/10524#discussion_r585326079
########## File path: indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/LagBasedAutoScaler.java ########## @@ -0,0 +1,242 @@ +/* + * 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.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 java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; + +public class LagBasedAutoScaler implements SupervisorTaskAutoScaler +{ + private static final EmittingLogger log = new EmittingLogger(LagBasedAutoScaler.class); + private final String dataSource; + private final CircularFifoQueue<Long> lagMetricsQueue; + private final ScheduledExecutorService lagComputationExec; + private final ScheduledExecutorService allocationExec; + private final SupervisorSpec spec; + private final SeekableStreamSupervisor supervisor; + private final LagBasedAutoScalerConfig lagBasedAutoScalerConfig; + + private static final ReentrantLock LOCK = new ReentrantLock(true); + + public LagBasedAutoScaler(SeekableStreamSupervisor supervisor, String dataSource, + LagBasedAutoScalerConfig autoScalerConfig, SupervisorSpec spec + ) + { + this.lagBasedAutoScalerConfig = autoScalerConfig; + final String supervisorId = StringUtils.format("Supervisor-%s", dataSource); + this.dataSource = dataSource; + final int slots = (int) (lagBasedAutoScalerConfig.getLagCollectionRangeMillis() / lagBasedAutoScalerConfig + .getLagCollectionIntervalMillis()) + 1; + this.lagMetricsQueue = new CircularFifoQueue<>(slots); + this.allocationExec = Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(supervisorId) + "-Allocation-%d"); + this.lagComputationExec = Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(supervisorId) + "-Computation-%d"); + this.spec = spec; + this.supervisor = supervisor; + } + + @Override + public void start() + { + Callable<Integer> scaleAction = () -> { + LOCK.lock(); + int desiredTaskCount = -1; + try { + desiredTaskCount = computeDesiredTaskCount(new ArrayList<>(lagMetricsQueue)); + + if (desiredTaskCount != -1) { + lagMetricsQueue.clear(); + } + } + catch (Exception ex) { + log.warn(ex, "Exception while computing desired task count for [%s]", dataSource); + } + finally { + LOCK.unlock(); + } + return desiredTaskCount; + }; + + lagComputationExec.scheduleAtFixedRate( + computeAndCollectLag(), + lagBasedAutoScalerConfig.getScaleActionStartDelayMillis(), // wait for tasks to start up + lagBasedAutoScalerConfig.getLagCollectionIntervalMillis(), + TimeUnit.MILLISECONDS + ); + allocationExec.scheduleAtFixedRate( + supervisor.buildDynamicAllocationTask(scaleAction), + lagBasedAutoScalerConfig.getScaleActionStartDelayMillis() + lagBasedAutoScalerConfig + .getLagCollectionRangeMillis(), Review comment: When `scaleActionStartDelayMillis` meets, lagComputationExec start to work to collect metrics. And allocationExec need to wait for another `lagCollectionRangeMillis` which means wait for lagComputationExec to collect enough lag metrics. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
