kfaraz commented on code in PR #18819:
URL: https://github.com/apache/druid/pull/18819#discussion_r2610661387
##########
extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaConsumerMonitor.java:
##########
@@ -181,4 +192,15 @@ public void stopAfterNextEmit()
{
stopAfterNext = true;
}
+
+ /**
+ * Average poll-to-idle ratio as reported by the Kafka consumer.
+ * A value of 0 represents that the consumer is never idle, i.e. always
consuming.
+ * A value of 1 represents that the consumer is always idle, i.e. not
receiving data.
+ * @return
Review Comment:
```suggestion
```
##########
embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.testing.embedded.indexing.autoscaler;
+
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.indexing.kafka.KafkaIndexTaskModule;
+import org.apache.druid.indexing.kafka.simulate.KafkaResource;
+import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus;
+import
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler;
+import
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScalerConfig;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.testing.embedded.EmbeddedBroker;
+import org.apache.druid.testing.embedded.EmbeddedClusterApis;
+import org.apache.druid.testing.embedded.EmbeddedCoordinator;
+import org.apache.druid.testing.embedded.EmbeddedDruidCluster;
+import org.apache.druid.testing.embedded.EmbeddedHistorical;
+import org.apache.druid.testing.embedded.EmbeddedIndexer;
+import org.apache.druid.testing.embedded.EmbeddedOverlord;
+import org.apache.druid.testing.embedded.EmbeddedRouter;
+import org.apache.druid.testing.embedded.indexing.MoreResources;
+import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.hamcrest.Matchers;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * Integration test for {@link
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler}.
+ * <p>
+ * Tests the autoscaler's ability to compute optimal task counts based on
partition count and cost metrics (lag and idle time).
+ */
+public class CostBasedAutoScalerIntegrationTest extends EmbeddedClusterTestBase
+{
+ private static final String TOPIC =
EmbeddedClusterApis.createTestDatasourceName();
+ private static final String EVENT_TEMPLATE =
"{\"timestamp\":\"%s\",\"dimension\":\"value%d\",\"metric\":%d}";
+ private static final int PARTITION_COUNT = 10;
+ private static final int INITIAL_TASK_COUNT = 10;
+
+ private final EmbeddedBroker broker = new EmbeddedBroker();
+ private final EmbeddedIndexer indexer = new EmbeddedIndexer();
+ private final EmbeddedOverlord overlord = new EmbeddedOverlord();
+ private final EmbeddedHistorical historical = new EmbeddedHistorical();
+ private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator();
+ private KafkaResource kafkaServer;
+
+ @Override
+ public EmbeddedDruidCluster createCluster()
+ {
+ final EmbeddedDruidCluster cluster =
EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper();
+
+ kafkaServer = new KafkaResource()
+ {
+ @Override
+ public void start()
+ {
+ super.start();
+ createTopicWithPartitions(TOPIC, PARTITION_COUNT);
+ produceRecordsToKafka(500);
+ }
+
+ @Override
+ public void stop()
+ {
+ deleteTopic(TOPIC);
+ super.stop();
+ }
+ };
+
+ // Increase worker capacity to handle more tasks
+ indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s")
+ .addProperty("druid.worker.capacity", "60");
+
+ overlord.addProperty("druid.indexer.task.default.context",
"{\"useConcurrentLocks\": true}")
+ .addProperty("druid.manager.segments.useIncrementalCache",
"ifSynced")
+ .addProperty("druid.manager.segments.pollDuration", "PT0.1s");
+
+ coordinator.addProperty("druid.manager.segments.useIncrementalCache",
"ifSynced");
+
+ cluster.useLatchableEmitter()
+ .addServer(coordinator)
+ .addServer(overlord)
+ .addServer(indexer)
+ .addServer(broker)
+ .addServer(historical)
+ .addExtension(KafkaIndexTaskModule.class)
+ .addCommonProperty("druid.monitoring.emissionPeriod", "PT0.5s")
+ .addResource(kafkaServer)
+ .addServer(new EmbeddedRouter());
+
+ return cluster;
+ }
+
+ @Test
+ @Timeout(45)
+ public void test_autoScaler_computesOptimalTaskCountAndProduceScaleDown()
+ {
+ final String superId = dataSource + "_super";
+
+ // Produce some amount of data to kafka, to trigger a 'scale down'
decision to 4 tasks.
+
+ final CostBasedAutoScalerConfig autoScalerConfig =
CostBasedAutoScalerConfig
+ .builder()
+ .enableTaskAutoScaler(true)
+ .taskCountMin(1)
+ .taskCountMax(100)
+ .taskCountStart(INITIAL_TASK_COUNT)
+ .metricsCollectionIntervalMillis(1000)
+ .scaleActionStartDelayMillis(1500)
+ .scaleActionPeriodMillis(1500)
+ .minTriggerScaleActionFrequencyMillis(3000)
+ // Weight configuration: strongly favor lag reduction over idle time
+ .lagWeight(0.9)
+ .idleWeight(0.1)
+ .build();
+
+ final KafkaSupervisorSpec kafkaSupervisorSpec =
createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig);
+
+ // Submit the supervisor
+ Assertions.assertEquals(superId,
cluster.callApi().postSupervisor(kafkaSupervisorSpec));
+
+ // Wait for the supervisor to be healthy and running
+ waitForSupervisorRunning(superId);
+
+ // Wait for autoscaler to emit optimalTaskCount metric indicating
scale-down
+ // We expect the optimal task count to 2
+ overlord.latchableEmitter().waitForEvent(
+ event ->
event.hasMetricName(CostBasedAutoScaler.OPTIMAL_TASK_COUNT_METRIC)
+ .hasValueMatching(Matchers.equalTo(4L))
+ );
+
+ // Suspend the supervisor
+
cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec());
+ }
+
+ private void waitForSupervisorRunning(String supervisorId)
+ {
+ int maxAttempts = 10;
+ int attempt = 0;
+ while (attempt < maxAttempts) {
+ SupervisorStatus status =
cluster.callApi().getSupervisorStatus(supervisorId);
+ if (status != null && "RUNNING".equals(status.getState()) &&
status.isHealthy()) {
+ return;
+ }
+ attempt++;
+ try {
+ Thread.sleep(1000);
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ }
Review Comment:
Instead of polling the API and sleeping, you can reduce `taskDuration` to
something very small (e.g. see `KafkaClusterMetricsTest`) and then wait for the
`task/run/time` metric or `ingest/count` metric to be emitted for a task of
this datasource/supervisor.
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/RecordSupplier.java:
##########
@@ -143,6 +142,15 @@ default Map<PartitionIdType, SequenceOffsetType>
getLatestSequenceNumbers(Set<St
throw new UnsupportedOperationException();
}
+ /**
+ * @return Kafka's `poll-idle-ratio-avg` an it's analog for Kinesis,
+ * required for correct autoscaler work
+ */
Review Comment:
Update pending.
##########
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. So, 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.
##########
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.
+ */
Review Comment:
We can avoid this javadoc as it does not add any new info. In fact, I think
we can get rid of this field altogether, if we do `collectMetrics` and
`supervisor.buildDynamicAllocationTask` in the same runnable.
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+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;
+
+ private static final Map<Integer, List<Integer>> FACTORS_CACHE = new
HashMap<>();
+
+ private final String dataSource;
+ /**
+ * 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 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;
+
+ 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);
+
+ this.currentMetrics = new AtomicReference<>(null);
+ 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 = () ->
computeOptimalTaskCount(currentMetrics);
+ Runnable onSuccessfulScale = () -> currentMetrics.set(null);
+
+ metricsCollectionExec.scheduleAtFixedRate(
+ collectMetrics(),
+ config.getMetricsCollectionIntervalMillis(),
+ config.getMetricsCollectionIntervalMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ scalingDecisionExec.scheduleAtFixedRate(
+ supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale,
emitter),
+ config.getScaleActionStartDelayMillis(),
+ config.getScaleActionPeriodMillis(),
+ TimeUnit.MILLISECONDS
Review Comment:
Sorry, I meant that we should not have the start delay, period and
collection interval as configs at all. Even with good defaults, unnecessary
configs only complicate admin work and require code to handle all possible
scenarios.
At most, maybe keep just one config `scaleActionPeriod` that can be
specified as an ISO period (e.g. `PT1M`) or something (mostly since you would
be using this in embedded tests). The other configs don't really add any value.
They are legacy configs in lag-based auto-scaler which we might as well avoid
adding in the new strategy.
##########
extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaConsumerMonitor.java:
##########
@@ -47,9 +48,11 @@ public class KafkaConsumerMonitor extends AbstractMonitor
private static final String PARTITION_TAG = "partition";
private static final String NODE_ID_TAG = "node-id";
+ private static final String POLL_IDLE_RATIO_METRIC_NAME =
"poll-idle-ratio-avg";
+
/**
* Kafka metric name -> Kafka metric descriptor. Taken from
- * https://kafka.apache.org/documentation/#consumer_fetch_monitoring.
+ * <a
href="https://kafka.apache.org/documentation/#consumer_fetch_monitoring">documentation</a>.
Review Comment:
```suggestion
* <a
href="https://kafka.apache.org/documentation/#consumer_fetch_monitoring">Kafka
documentation</a>.
```
##########
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),
+ config.getScaleActionStartDelayMillis(),
+ config.getScaleActionPeriodMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ log.info(
+ "CostBasedAutoScaler started for dataSource [%s]: collecting metrics
every [%d]ms, "
+ + "evaluating scaling every [%d]ms",
+ supervisorId,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getScaleActionPeriodMillis()
+ );
+ }
+
+ @Override
+ public void stop()
+ {
+ autoscalerExecutor.shutdownNow();
+ log.info("CostBasedAutoScaler stopped for dataSource [%s]", supervisorId);
+ }
+
+ @Override
+ public void reset()
+ {
+ currentMetrics.set(null);
+ }
+
+ private void collectMetrics()
+ {
+ if (spec.isSuspended()) {
+ log.debug("Supervisor [%s] is suspended, skipping a metrics collection",
supervisorId);
+ return;
+ }
+
+ final LagStats lagStats = supervisor.computeLagStats();
+ if (lagStats == null) {
+ log.debug("Lag stats unavailable for dataSource [%s], skipping
collection", supervisorId);
+ return;
+ }
+
+ final int currentTaskCount = supervisor.getIoConfig().getTaskCount();
+ final int partitionCount = supervisor.getPartitionCount();
+ final double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+ currentMetrics.set(
+ new CostMetrics(
+ lagStats.getAvgLag(),
+ currentTaskCount,
+ partitionCount,
+ pollIdleRatio
+ )
+ );
+
+ log.debug("Collected metrics for dataSource [%s]", supervisorId);
+ }
+
+ /**
+ * @return optimal task count, or -1 if no scaling action needed
+ */
+ public int computeOptimalTaskCount(AtomicReference<CostMetrics>
currentMetricsRef)
+ {
+ final CostMetrics metrics = currentMetricsRef.get();
+ if (metrics == null) {
+ log.debug("No metrics available yet for dataSource [%s]", supervisorId);
Review Comment:
Please update all the log lines in this class to contain the string
`supervisorId` instead of `dataSource`.
```suggestion
log.debug("No metrics available yet for supervisorId[%s]",
supervisorId);
```
##########
indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java:
##########
@@ -69,7 +69,7 @@ public enum IngestionMode
{
REPLACE, // replace with tombstones
APPEND, // append to existing segments
- REPLACE_LEGACY, // original replace, it does not replace existing data for
empty time chunks in input intervals
+ REPLACE_LEGACY, // original 'replace', it does not replace existing data
for empty time chunks in input intervals
Review Comment:
Minor suggestion:
Can we revert this so that we don't touch `AbstractTask` in this PR as this
class is unrelated to the changes here?
Please consider reverting formatting changes which are large or which touch
files that are not already a part of this PR.
Please feel free to create a separate patch for any typos/doc updates.
##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/autoscaler/CostBasedAutoScaler.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.Collections;
+import java.util.HashMap;
+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;
+
+ private static final Map<Integer, List<Integer>> FACTORS_CACHE = new
HashMap<>();
+
+ private final String dataSource;
+ /**
+ * 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 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;
+
+ 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);
+
+ this.currentMetrics = new AtomicReference<>(null);
+ 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 = () ->
computeOptimalTaskCount(currentMetrics);
+ Runnable onSuccessfulScale = () -> currentMetrics.set(null);
+
+ metricsCollectionExec.scheduleAtFixedRate(
+ collectMetrics(),
+ config.getMetricsCollectionIntervalMillis(),
+ config.getMetricsCollectionIntervalMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ scalingDecisionExec.scheduleAtFixedRate(
+ supervisor.buildDynamicAllocationTask(scaleAction, onSuccessfulScale,
emitter),
+ config.getScaleActionStartDelayMillis(),
+ config.getScaleActionPeriodMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ log.info(
+ "CostBasedAutoScaler started for dataSource [%s]: collecting metrics
every [%d]ms, "
Review Comment:
```suggestion
"CostBasedAutoScaler started for supervisorId[%s]: collecting
metrics every [%d]ms, "
```
##########
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),
+ config.getScaleActionStartDelayMillis(),
+ config.getScaleActionPeriodMillis(),
+ TimeUnit.MILLISECONDS
+ );
+
+ log.info(
+ "CostBasedAutoScaler started for dataSource [%s]: collecting metrics
every [%d]ms, "
+ + "evaluating scaling every [%d]ms",
+ supervisorId,
+ config.getMetricsCollectionIntervalMillis(),
+ config.getScaleActionPeriodMillis()
+ );
+ }
+
+ @Override
+ public void stop()
+ {
+ autoscalerExecutor.shutdownNow();
+ log.info("CostBasedAutoScaler stopped for dataSource [%s]", supervisorId);
+ }
+
+ @Override
+ public void reset()
+ {
+ currentMetrics.set(null);
+ }
+
+ private void collectMetrics()
+ {
+ if (spec.isSuspended()) {
+ log.debug("Supervisor [%s] is suspended, skipping a metrics collection",
supervisorId);
+ return;
+ }
+
+ final LagStats lagStats = supervisor.computeLagStats();
+ if (lagStats == null) {
+ log.debug("Lag stats unavailable for dataSource [%s], skipping
collection", supervisorId);
+ return;
+ }
+
+ final int currentTaskCount = supervisor.getIoConfig().getTaskCount();
+ final int partitionCount = supervisor.getPartitionCount();
+ final double pollIdleRatio = supervisor.getPollIdleRatioMetric();
+
+ currentMetrics.set(
+ new CostMetrics(
+ lagStats.getAvgLag(),
+ currentTaskCount,
+ partitionCount,
+ pollIdleRatio
+ )
+ );
+
+ log.debug("Collected metrics for dataSource [%s]", supervisorId);
+ }
+
+ /**
+ * @return optimal task count, or -1 if no scaling action needed
+ */
+ public int computeOptimalTaskCount(AtomicReference<CostMetrics>
currentMetricsRef)
Review Comment:
Nit: You could just pass the `CostMetrics` to this method rather than the
atomic reference.
--
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]