-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Introduce cost-based tasks autoscaler for streaming ingestion #18819
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
23 commits
Select commit
Hold shift + click to select a range
b7703c8
Cost-based autoscaler: first raw version
Fly-Style 704ed4c
Use single atomic ref instead of locking, fix tests, cleanup
Fly-Style 1fe225d
WIP: integration test
Fly-Style 4a18380
Use kinda correct metric for idle-ratio, fix metrics emission
Fly-Style e2aa55a
Finish the integration test; introduce idle metrics
Fly-Style 9dabb99
Review comments pass: WIP
Fly-Style e5b40c7
Address review comments
Fly-Style 9bc5107
Addressing review comments - 2
Fly-Style f81b7b8
Add a new HTTP endpoint on task side and call on supervisor side for …
Fly-Style db1254b
Use http call to get poll-idle-ratio-avg metric
Fly-Style 33c9549
Remove factors cache as more dangerous option, just leaving the compu…
Fly-Style 1e5281b
Adjust WeightedCostFunction implementation with focus on ideal idle r…
Fly-Style fdbe06f
Extract stats via /rowStats call, remove /metrics call
Fly-Style 9ecafee
Enable scaleUp integration test
Fly-Style 3cda48a
Address review comments - 3
Fly-Style 922a7c6
Remove obsolete comment in tests
Fly-Style 788e8e6
Update extensions-core/kafka-indexing-service/src/main/java/org/apach…
Fly-Style 1647ee9
Update extensions-core/kafka-indexing-service/src/main/java/org/apach…
Fly-Style 286aa4e
Adjust computeValidTaskCounts implementation
Fly-Style 3fed728
Apply suggestions from code review
Fly-Style 161fdf3
Adjust cost function: use direct lag values, not normalized ones
Fly-Style e8f8247
Change weight function to linear and switch cost from abstract to est…
Fly-Style dbb264c
Remove unused variable taskCountDiff
kfaraz File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
267 changes: 267 additions & 0 deletions
267
...apache/druid/testing/embedded/indexing/autoscaler/CostBasedAutoScalerIntegrationTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,267 @@ | ||
| /* | ||
| * 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.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.query.DruidMetrics; | ||
| 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.joda.time.Seconds; | ||
| 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.concurrent.Executors; | ||
| import java.util.stream.Collectors; | ||
| import java.util.stream.IntStream; | ||
|
|
||
| import static org.apache.druid.indexing.seekablestream.supervisor.autoscaler.CostBasedAutoScaler.OPTIMAL_TASK_COUNT_METRIC; | ||
|
|
||
| /** | ||
| * Integration test for {@link 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 = 50; | ||
|
|
||
| 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, 1); | ||
| } | ||
|
|
||
| @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() | ||
| .useDefaultTimeoutForLatchableEmitter(120) | ||
| .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"; | ||
| final int initialTaskCount = 10; | ||
|
|
||
| final CostBasedAutoScalerConfig autoScalerConfig = CostBasedAutoScalerConfig | ||
| .builder() | ||
| .enableTaskAutoScaler(true) | ||
| .taskCountMin(1) | ||
| .taskCountMax(100) | ||
| .taskCountStart(initialTaskCount) | ||
| .scaleActionPeriodMillis(1500) | ||
| .minTriggerScaleActionFrequencyMillis(3000) | ||
| // Weight configuration: strongly favor lag reduction over idle time | ||
| .lagWeight(0.9) | ||
| .idleWeight(0.1) | ||
| .build(); | ||
|
|
||
| final KafkaSupervisorSpec spec = createKafkaSupervisorWithAutoScaler(superId, autoScalerConfig, initialTaskCount); | ||
|
|
||
| // Submit the supervisor | ||
| Assertions.assertEquals(superId, cluster.callApi().postSupervisor(spec)); | ||
|
|
||
| // Wait for the supervisor to be healthy and running | ||
| overlord.latchableEmitter() | ||
| .waitForEvent(event -> event.hasMetricName("task/run/time") | ||
| .hasDimension(DruidMetrics.DATASOURCE, dataSource)); | ||
|
|
||
| // Wait for autoscaler to emit optimalTaskCount metric indicating scale-down | ||
| // We expect the optimal task count to 4 | ||
| overlord.latchableEmitter().waitForEvent( | ||
| event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC) | ||
| .hasValueMatching(Matchers.equalTo(6L)) | ||
| ); | ||
|
|
||
| // Suspend the supervisor | ||
| cluster.callApi().postSupervisor(spec.createSuspendedSpec()); | ||
| } | ||
|
|
||
| @Test | ||
| @Timeout(125) | ||
| public void test_autoScaler_computesOptimalTaskCountAndProducesScaleUp() | ||
| { | ||
| final String superId = dataSource + "_super_scaleup"; | ||
|
|
||
| // Start with a low task count (1 task for 50 partitions) and produce a large amount of data | ||
| // to create lag pressure and low idle ratio, which should trigger a scale-up decision. | ||
| // With the ideal idle range [0.2, 0.6], a single overloaded task will have idle < 0.2, | ||
| // triggering the cost function to recommend more tasks. | ||
| final int lowInitialTaskCount = 1; | ||
|
|
||
| // Produce additional records to create a backlog / lag | ||
| // This ensures tasks are busy processing (low idle ratio) | ||
| Executors.newSingleThreadExecutor().submit(() -> produceRecordsToKafka(500_000, 20)); | ||
|
|
||
| // These values were carefully handpicked to allow that test to pass in a stable manner. | ||
| final CostBasedAutoScalerConfig autoScalerConfig = CostBasedAutoScalerConfig | ||
| .builder() | ||
| .enableTaskAutoScaler(true) | ||
| .taskCountMin(1) | ||
| .taskCountMax(50) | ||
| .taskCountStart(lowInitialTaskCount) | ||
| .scaleActionPeriodMillis(500) | ||
| .minTriggerScaleActionFrequencyMillis(1000) | ||
| .lagWeight(0.2) | ||
| .idleWeight(0.8) | ||
| .build(); | ||
|
|
||
| final KafkaSupervisorSpec kafkaSupervisorSpec = createKafkaSupervisorWithAutoScaler( | ||
| superId, | ||
| autoScalerConfig, | ||
| lowInitialTaskCount | ||
| ); | ||
|
|
||
| // Submit the supervisor | ||
| Assertions.assertEquals(superId, cluster.callApi().postSupervisor(kafkaSupervisorSpec)); | ||
|
|
||
| // Wait for the supervisor to be healthy and running | ||
| overlord.latchableEmitter() | ||
| .waitForEvent(event -> event.hasMetricName("task/run/time") | ||
| .hasDimension(DruidMetrics.DATASOURCE, dataSource)); | ||
|
|
||
| // With 50 partitions and high lag creating a low idle ratio (< 0.2), | ||
| // the cost function must recommend scaling up to at least 2 tasks. | ||
| overlord.latchableEmitter().waitForEvent( | ||
| event -> event.hasMetricName(OPTIMAL_TASK_COUNT_METRIC) | ||
| .hasValueMatching(Matchers.greaterThan(1L)) | ||
| ); | ||
|
|
||
| // Suspend the supervisor | ||
| cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); | ||
| } | ||
|
|
||
| private void produceRecordsToKafka(int recordCount, int iterations) | ||
| { | ||
| int recordCountPerSlice = recordCount / iterations; | ||
| int counter = 0; | ||
| for (int i = 0; i < iterations; i++) { | ||
| DateTime timestamp = DateTime.now(DateTimeZone.UTC); | ||
| List<ProducerRecord<byte[], byte[]>> records = IntStream | ||
| .range(counter, counter + recordCountPerSlice) | ||
| .mapToObj(k -> new ProducerRecord<byte[], byte[]>( | ||
| TOPIC, | ||
| k % PARTITION_COUNT, | ||
| null, | ||
| StringUtils.format(EVENT_TEMPLATE, timestamp, k, k) | ||
| .getBytes(StandardCharsets.UTF_8) | ||
| ) | ||
| ) | ||
| .collect(Collectors.toList()); | ||
|
|
||
| kafkaServer.produceRecordsToTopic(records); | ||
| try { | ||
| Thread.sleep(100L); | ||
| counter += recordCountPerSlice; | ||
| } | ||
| catch (InterruptedException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private KafkaSupervisorSpec createKafkaSupervisorWithAutoScaler( | ||
| String supervisorId, | ||
| CostBasedAutoScalerConfig autoScalerConfig, | ||
| int taskCount | ||
| ) | ||
| { | ||
| return MoreResources.Supervisor.KAFKA_JSON | ||
| .get() | ||
| .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) | ||
| .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(100)) | ||
| .withIoConfig( | ||
| ioConfig -> ioConfig | ||
| .withConsumerProperties(kafkaServer.consumerProperties()) | ||
| .withTaskCount(taskCount) | ||
| .withTaskDuration(Seconds.THREE.toPeriod()) | ||
| .withAutoScalerConfig(autoScalerConfig) | ||
| ) | ||
| .withId(supervisorId) | ||
| .build(dataSource, TOPIC); | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.