-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Implementing perpetually running tasks for Streaming Ingestion #18466
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
Open
uds5501
wants to merge
49
commits into
apache:master
Choose a base branch
from
uds5501:supervisor-reassignment-rollovers
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
49 commits
Select commit
Hold shift + click to select a range
8d683ab
task partition reassignment API design
uds5501 5fe8a1d
Introduce updateConfig API
uds5501 16960c0
Update config request to just use generic objects
uds5501 3fd66cb
Add configs to check whether perpetually running tasks should be used…
uds5501 b974ec8
Make partition assignment sequential
uds5501 26379c5
Implement supervisor changes to handle perpetual tasks
uds5501 27dddae
Fix compilation and checkstyle issues
uds5501 884487d
Merge branch 'master' of github.com:apache/druid into supervisor-reas…
uds5501 aa67322
Fix kinesis compilation failures
uds5501 a57d3eb
Working compilation
uds5501 a690e17
Add embedded tests
uds5501 a85c85e
Get embedded tests working
uds5501 7060e63
Push embedded tests
uds5501 870ef5b
Perform parsing in task runner
uds5501 7b5524b
Emit events when config is updated
uds5501 77193e0
WIP: Attempting to fix failing latch
uds5501 4b51c14
Fix latches and clean up maven issues
uds5501 dcd3549
Use atomic booleans for coordination b/w multiple triggers of auto sc…
uds5501 673f114
Fix range based partition assignment logic
uds5501 160dd9f
WIP: Fix the latch issue, use settable futures instead.
uds5501 30330c5
WIP: Additional logging
uds5501 85e45fc
Merge branch 'master' of github.com:apache/druid into supervisor-reas…
uds5501 0adb18b
Fix build issue
uds5501 c332e54
Fix checkpoint and seeking issues
uds5501 6135cce
Fix perpetually running task config send
uds5501 653ef63
WIP: Continous publish and handoffs for perpetual tasks
uds5501 a6c949c
WIP: Save updated task config in taskStorage, todo: save it via taksQ…
uds5501 2e66e4d
Fix succeeding tasks issues
uds5501 b3a1cf6
revert separate thread for scheduling publishes
uds5501 54dbd68
Fix the checkpoint <> task discovery race condition
uds5501 733064b
First embedded test success run
uds5501 5907491
Make maven happy
uds5501 3852d8f
Skip rollovers
uds5501 512a853
Merge branch 'master' of github.com:apache/druid into supervisor-reas…
uds5501 e813699
Fix failing embedded test
uds5501 6874292
WIP - improve coverage
uds5501 bad6b7f
WIP - improve coverage 2
uds5501 1dab564
Fix tests and implcitly turn on release locks on handoff to true
uds5501 2054c06
Fix scale in issues
uds5501 290bc6b
Scale down the embedded tests
uds5501 cacabbf
See if removing router fixes js error
uds5501 819973d
Attempt clearing console from the deps
uds5501 6c11f9b
Should fix flaky embedded test
uds5501 5d09806
Remove complaining ioConfig and add kafka supervisor test
uds5501 3392bda
Address review comments related to Index task runner
uds5501 0aa8439
Stop persisting task configs and address other comments
uds5501 8ce15d2
Rename a few tests
uds5501 f0997ba
Add new embedded tests, fix no data pushed bug, revert some refactors
uds5501 799c547
Complete all embedded test additions
uds5501 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
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
227 changes: 227 additions & 0 deletions
227
...ts/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.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,227 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| import org.apache.druid.data.input.impl.TimestampSpec; | ||
| import org.apache.druid.guice.ClusterTestingModule; | ||
| 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.kafka.supervisor.LagBasedAutoScalerConfigBuilder; | ||
| import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; | ||
| import org.apache.druid.testing.cluster.overlord.FaultyLagAggregator; | ||
| 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.junit5.EmbeddedClusterTestBase; | ||
| import org.joda.time.Period; | ||
| import org.junit.jupiter.api.Assertions; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.api.Timeout; | ||
|
|
||
| /** | ||
| * Embedded test to verify task scaling behaviour of {@code KafkaSupervisor} ingesting from a custom kafka topic. | ||
| */ | ||
| @SuppressWarnings("resource") | ||
| public class KafkaTaskAutoScalingTest extends EmbeddedClusterTestBase | ||
| { | ||
| private static final String TOPIC = EmbeddedClusterApis.createTestDatasourceName(); | ||
|
|
||
| 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, 10); | ||
| } | ||
|
|
||
| @Override | ||
| public void stop() | ||
| { | ||
| deleteTopic(TOPIC); | ||
| super.stop(); | ||
| } | ||
| }; | ||
|
|
||
| indexer.addProperty("druid.worker.capacity", "10"); | ||
| overlord.addProperty("druid.server.http.numThreads", "50") | ||
| .addProperty("druid.unsafe.cluster.testing", "true"); | ||
|
|
||
| cluster.addExtension(KafkaIndexTaskModule.class) | ||
| .addExtension(ClusterTestingModule.class) | ||
| .addResource(kafkaServer) | ||
| .addServer(coordinator) | ||
| .addServer(overlord) | ||
| .addServer(indexer) | ||
| .addServer(broker) | ||
| .addServer(historical) | ||
| .addServer(new EmbeddedRouter()) | ||
| .useLatchableEmitter(); | ||
|
|
||
| return cluster; | ||
| } | ||
|
|
||
| @Test | ||
| @Timeout(50) | ||
| public void test_supervisorTasksScalesOutAndScalesIn_withPersistentTasksAndAutoScaler() throws Exception | ||
| { | ||
| final int initialTaskCount = 1; | ||
| final int taskCountMax = 3; | ||
| final int taskCountMin = 1; | ||
|
|
||
| AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() | ||
| .withLagCollectionIntervalMillis(500) | ||
| .withLagCollectionRangeMillis(1000) | ||
| .withEnableTaskAutoScaler(true) | ||
| .withScaleActionPeriodMillis(2000) | ||
| .withScaleActionStartDelayMillis(1000) | ||
| .withScaleOutThreshold(100) | ||
| .withScaleInThreshold(1) | ||
| .withTaskCountMin(taskCountMin) | ||
| .withTaskCountMax(taskCountMax) | ||
| .withTaskCountStart(initialTaskCount) | ||
| .withTriggerScaleOutFractionThreshold(0.001) | ||
| .withTriggerScaleInFractionThreshold(0.01) | ||
| .withScaleOutStep(1) | ||
| .withScaleInStep(1) | ||
| .withMinTriggerScaleActionFrequencyMillis(3000) | ||
| .withStopTaskCountRatio(1.0) | ||
| .build(); | ||
|
|
||
| String supervisorId = dataSource + "_scale_out_and_in_to_zero"; | ||
| String controllerId = "artificial-lag-controller"; | ||
| final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpecWithControlledLag( | ||
| supervisorId, | ||
| controllerId, | ||
| initialTaskCount, | ||
| autoScalerConfig, | ||
| true | ||
| ); | ||
|
|
||
| Assertions.assertEquals( | ||
| supervisorId, | ||
| cluster.callApi().postSupervisor(kafkaSupervisorSpec) | ||
| ); | ||
|
|
||
| FaultyLagAggregator.injectLag(controllerId, 100000L); | ||
|
|
||
| overlord.latchableEmitter().waitForEventAggregate( | ||
| event -> event.hasMetricName("task/autoScaler/scaleActionTime"), | ||
| agg -> agg.hasCountAtLeast(2) | ||
| ); | ||
|
|
||
| Thread.sleep(5000); | ||
|
|
||
| String runningTaskCountResult = cluster.runSql( | ||
| "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", | ||
| dataSource | ||
| ); | ||
| int runningTasks = Integer.parseInt(runningTaskCountResult); | ||
|
|
||
| Assertions.assertEquals( | ||
| taskCountMax, | ||
| runningTasks, | ||
| String.format("Expected %d tasks to be running after scale up, but found %d", taskCountMax, runningTasks) | ||
| ); | ||
|
|
||
| FaultyLagAggregator.injectLag(controllerId, 0L); | ||
|
|
||
| overlord.latchableEmitter().waitForEventAggregate( | ||
| event -> event.hasMetricName("task/autoScaler/scaleActionTime"), | ||
| agg -> agg.hasCountAtLeast(2) | ||
| ); | ||
| Thread.sleep(10000); | ||
|
|
||
| // ensure everything has shutdown. | ||
| runningTaskCountResult = cluster.runSql( | ||
| "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", | ||
| dataSource | ||
| ); | ||
| String successTaskCountResult = cluster.runSql( | ||
| "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'SUCCESS'", | ||
| dataSource | ||
| ); | ||
| runningTasks = Integer.parseInt(runningTaskCountResult); | ||
| final int successTasks = Integer.parseInt(successTaskCountResult); | ||
|
|
||
| Assertions.assertEquals( | ||
| taskCountMin, | ||
| runningTasks, | ||
| String.format("Expected %d task to be running after scale down, but found %d", runningTasks, taskCountMin) | ||
| ); | ||
|
|
||
| int shutDownTasksExpected = taskCountMax - taskCountMin; | ||
|
|
||
| Assertions.assertTrue( | ||
| successTasks >= shutDownTasksExpected, | ||
| String.format("Expected at least %d task to be successfully completed, but found %d", shutDownTasksExpected, successTasks) | ||
| ); | ||
|
|
||
| // Cleanup | ||
| FaultyLagAggregator.clearAllInjectedLag(); | ||
| cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); | ||
| } | ||
|
|
||
| private KafkaSupervisorSpec createSupervisorSpecWithControlledLag( | ||
| String supervisorId, | ||
| String controllerId, | ||
| int taskCount, | ||
| AutoScalerConfig autoScalerConfig, | ||
| boolean usePersistentTasks | ||
| ) | ||
| { | ||
| return MoreResources.Supervisor.KAFKA_JSON | ||
| .get() | ||
| .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) | ||
| .withTuningConfig(tuningConfig -> tuningConfig | ||
| .withMaxRowsPerSegment(1000) | ||
| .withWorkerThreads(10)) | ||
| .withIoConfig( | ||
| ioConfig -> ioConfig | ||
| .withConsumerProperties(kafkaServer.consumerProperties()) | ||
| .withTaskCount(taskCount) | ||
| .withTaskDuration(Period.millis(30000)) | ||
| .withAutoScalerConfig(autoScalerConfig) | ||
| .withLagAggregator(new FaultyLagAggregator(1, controllerId)) | ||
| ) | ||
| .withId(supervisorId) | ||
| .withUsePersistentTasks(usePersistentTasks) | ||
| .build(dataSource, TOPIC); | ||
| } | ||
| } |
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please revert all the changes to this file.