diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index 5a977b861e81..155c8a6beb4b 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -70,6 +70,7 @@ public class KafkaClusterMetricsTest extends EmbeddedClusterTestBase private final EmbeddedOverlord overlord = new EmbeddedOverlord(); private final EmbeddedHistorical historical = new EmbeddedHistorical(); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator(); + private KafkaResource kafkaServer; @Override @@ -98,8 +99,10 @@ public void stop() }; indexer.addProperty("druid.segment.handoff.pollDuration", "PT0.1s") + .addProperty("druid.server.http.numThreads", "30") .addProperty("druid.worker.capacity", "10"); overlord.addProperty("druid.indexer.task.default.context", "{\"useConcurrentLocks\": true}") + .addProperty("druid.server.http.numThreads", "50") .addProperty("druid.manager.segments.useIncrementalCache", "ifSynced") .addProperty("druid.manager.segments.pollDuration", "PT0.1s") .addProperty("druid.manager.segments.killUnused.enabled", "true") @@ -391,7 +394,10 @@ private KafkaSupervisorSpec createKafkaSupervisor( return MoreResources.Supervisor.KAFKA_JSON .get() .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) - .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(maxRowsPerSegment)) + .withTuningConfig(tuningConfig -> tuningConfig + .withMaxRowsPerSegment(maxRowsPerSegment) + .withWorkerThreads(10) + .withReleaseLocksOnHandoff(true)) .withIoConfig( ioConfig -> ioConfig .withConsumerProperties(kafkaServer.consumerProperties()) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.java new file mode 100644 index 000000000000..2c4ae7d7007f --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskAutoScalingTest.java @@ -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); + } +} diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java new file mode 100644 index 000000000000..290c20d18fea --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaTaskScalingTest.java @@ -0,0 +1,259 @@ +/* + * 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.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.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 KafkaTaskScalingTest 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"); + cluster.addExtension(KafkaIndexTaskModule.class) + .addResource(kafkaServer) + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(broker) + .addServer(historical) + .addServer(new EmbeddedRouter()) + .useLatchableEmitter(); + + return cluster; + } + + @Test + @Timeout(10) + public void test_supervisorTasksFinish_withNoDataAndShortTaskDuration() + { + final int taskCount = 3; + + final String supervisorId = dataSource + "_short_tasks"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpec(supervisorId, taskCount, null, false); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + try { + Thread.sleep(2000); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + Assertions.fail("Test interrupted"); + } + + final String successTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'SUCCESS'", + dataSource + ); + final int successfulTasks = Integer.parseInt(successTaskCountResult); + + Assertions.assertEquals( + taskCount, + successfulTasks, + String.format("Expected all %d tasks to succeed, but only %d succeeded", taskCount, successfulTasks) + ); + + cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); + } + + @Test + @Timeout(20) + public void test_supervisorTasksDontFinish_withPersistentTasks() + { + final int taskCount = 3; + + final String supervisorId = dataSource + "_persistent_short_tasks"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpec(supervisorId, taskCount, null, true); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + try { + Thread.sleep(10000); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + Assertions.fail("Test interrupted"); + } + + final String runningTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", + dataSource + ); + final int runningTasks = Integer.parseInt(runningTaskCountResult); + + Assertions.assertEquals( + taskCount, + runningTasks, + String.format("Expected all %d tasks to be running, but only %d were found running", taskCount, runningTasks) + ); + + cluster.callApi().postSupervisor(kafkaSupervisorSpec.createSuspendedSpec()); + } + + @Test + @Timeout(20) + public void test_supervisorTasksScalesIn_withPersistentTasksAndAutoScaler() + { + final int initialTaskCount = 3; + final int taskCountMin = 1; + + AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfigBuilder() + .withLagCollectionIntervalMillis(500) + .withLagCollectionRangeMillis(1000) + .withEnableTaskAutoScaler(true) + .withScaleActionPeriodMillis(5000) + .withScaleActionStartDelayMillis(5000) + .withScaleOutThreshold(10000) + .withScaleInThreshold(1) + .withTaskCountMin(taskCountMin) + .withTriggerScaleOutFractionThreshold(0.9) + .withTriggerScaleInFractionThreshold(0.001) + .withTaskCountMax(initialTaskCount) + .withTaskCountStart(initialTaskCount) + .withScaleOutStep(0) + .withScaleInStep(1) + .withMinTriggerScaleActionFrequencyMillis(1000) + .withStopTaskCountRatio(1.0) + .build(); + + String supervisorId = dataSource + "_persistent_autoscale_tasks"; + final KafkaSupervisorSpec kafkaSupervisorSpec = createSupervisorSpec( + supervisorId, + initialTaskCount, + autoScalerConfig, + true + ); + + Assertions.assertEquals( + supervisorId, + cluster.callApi().postSupervisor(kafkaSupervisorSpec) + ); + + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("task/autoScaler/scaleActionTime"), + agg -> agg.hasCountAtLeast(2) + ); + + try { + Thread.sleep(2000); // Wait for a few seconds for the tasks to scale in + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + Assertions.fail("Test interrupted"); + } + + final String runningTaskCountResult = cluster.runSql( + "SELECT COUNT(*) FROM sys.tasks WHERE datasource = '%s' AND status = 'RUNNING'", + dataSource + ); + + final int runningTasks = Integer.parseInt(runningTaskCountResult); + + Assertions.assertEquals( + taskCountMin, + runningTasks, + String.format("Expected all %d tasks to be running, but only %d were found running", taskCountMin, runningTasks) + ); + } + + private KafkaSupervisorSpec createSupervisorSpec( + String supervisorId, + 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(500)) + .withAutoScalerConfig(autoScalerConfig) + ) + .withId(supervisorId) + .withUsePersistentTasks(usePersistentTasks) + .build(dataSource, TOPIC); + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java index 4aceebe37be7..4fe8cdae2ab8 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.utils.RuntimeInfo; @@ -67,7 +68,9 @@ public RabbitStreamIndexTask( tuningConfig, ioConfig, context, - getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE) + getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + null, + null ); this.configMapper = configMapper; @@ -122,6 +125,21 @@ protected RabbitStreamRecordSupplier newTaskRecordSupplier() ); } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return new RabbitStreamIndexTask( + getId(), + getSupervisorId(), + getTaskResource(), + getDataSchema(), + getTuningConfig(), + (RabbitStreamIndexTaskIOConfig) newIoConfig, + getContext(), + configMapper + ); + } + @Override @JsonProperty public RabbitStreamIndexTaskTuningConfig getTuningConfig() diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java index 2ced013ef5bc..b4a5a14b8440 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java @@ -402,6 +402,17 @@ protected Map getLatestSequencesFromStream() return latestSequenceFromStream != null ? latestSequenceFromStream : new HashMap<>(); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ) + { + throw new UnsupportedOperationException("not implemented"); + } + @Override protected String baseTaskName() { diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java index 4a445f6f1c11..c962782d7d9c 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java @@ -82,7 +82,8 @@ public RabbitStreamSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig); + supervisorStateManagerConfig, + null); } @Override diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java index a2667026fffd..ea2bc7b25e1a 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java @@ -219,7 +219,7 @@ public String toString() } @Override - public RabbitStreamIndexTaskTuningConfig convertToTaskTuningConfig() + public RabbitStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new RabbitStreamIndexTaskTuningConfig( getAppendableIndexSpec(), diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index f19ac81a85b1..6f77b99434cd 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.security.AuthorizationUtils; @@ -69,6 +70,8 @@ public KafkaIndexTask( @JsonProperty("tuningConfig") KafkaIndexTaskTuningConfig tuningConfig, @JsonProperty("ioConfig") KafkaIndexTaskIOConfig ioConfig, @JsonProperty("context") Map context, + @JsonProperty("isPerpetuallyRunning") @Nullable Boolean isPerpetuallyRunning, + @JsonProperty("supervisorSpecVersion") @Nullable String supervisorSpecVersion, @JacksonInject ObjectMapper configMapper ) { @@ -80,7 +83,9 @@ public KafkaIndexTask( tuningConfig, ioConfig, context, - getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE) + getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + isPerpetuallyRunning, + supervisorSpecVersion ); this.configMapper = configMapper; @@ -132,6 +137,23 @@ protected KafkaRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) } } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return new KafkaIndexTask( + getId(), + getSupervisorId(), + getTaskResource(), + getDataSchema(), + getTuningConfig(), + (KafkaIndexTaskIOConfig) newIoConfig, + getContext(), + isPerpetuallyRunning(), + getSupervisorSpecVersion(), + configMapper + ); + } + @Override @JsonProperty public KafkaIndexTaskTuningConfig getTuningConfig() diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 68bdb3fb4050..ef627a110c58 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -226,6 +226,70 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ) + { + log.info("Creating updated task IO config for task group [%s]", existingTaskGroup.getId()); + Map startingSequences = new HashMap<>(); + Set exclusiveStartSequenceNumberPartitions = new HashSet<>(); + + for (KafkaTopicPartition partition : partitions) { + Long offset = Math.max( + latestTaskOffsetsOnPause.getOrDefault(partition, 0L), + latestCommittedOffsets.getOrDefault(partition, 0L) + ); + startingSequences.put(partition, offset); + } + + SeekableStreamStartSequenceNumbers startSequenceNumbers = + new SeekableStreamStartSequenceNumbers<>( + spec.getIoConfig().getStream(), + startingSequences, + exclusiveStartSequenceNumberPartitions + ); + + // For end sequences, use NOT_SET to indicate open-ended reading + Map endingSequences = new HashMap<>(); + for (KafkaTopicPartition partition : partitions) { + endingSequences.put(partition, END_OF_PARTITION); + } + + SeekableStreamEndSequenceNumbers endSequenceNumbers = + new SeekableStreamEndSequenceNumbers<>( + spec.getIoConfig().getStream(), + endingSequences + ); + + log.info( + "Created updated IOConfig with starting sequences [%s] for partitions [%s]", + startingSequences, partitions + ); + + // Create the updated IOConfig + return new KafkaIndexTaskIOConfig( + existingTaskGroup.getId(), + existingTaskGroup.getBaseSequenceName(), + null, + null, + startSequenceNumbers, + endSequenceNumbers, + spec.getIoConfig().getConsumerProperties(), + spec.getIoConfig().getPollTimeout(), + true, + existingTaskGroup.getMinimumMessageTime(), + existingTaskGroup.getMaximumMessageTime(), + spec.getIoConfig().getInputFormat(), + spec.getIoConfig().getConfigOverrides(), + spec.getIoConfig().isMultiTopic(), + spec.getIoConfig().getTaskDuration().getStandardMinutes() + ); + } + @Override protected List> createIndexTasks( int replicas, @@ -252,6 +316,8 @@ protected List context, @JsonProperty("suspended") Boolean suspended, + @JsonProperty("usePersistentTasks") @Nullable Boolean usePersistentTasks, @JacksonInject TaskStorage taskStorage, @JacksonInject TaskMaster taskMaster, @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, @@ -89,7 +90,8 @@ public KafkaSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig + supervisorStateManagerConfig, + usePersistentTasks ); } @@ -161,6 +163,7 @@ protected KafkaSupervisorSpec toggleSuspend(boolean suspend) getIoConfig(), getContext(), suspend, + usePersistentTasks(), taskStorage, taskMaster, indexerMetadataStorageCoordinator, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index c4a21674d301..1a177128e6d4 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -213,7 +213,7 @@ public String toString() } @Override - public KafkaIndexTaskTuningConfig convertToTaskTuningConfig() + public KafkaIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new KafkaIndexTaskTuningConfig( getAppendableIndexSpec(), @@ -237,7 +237,7 @@ public KafkaIndexTaskTuningConfig convertToTaskTuningConfig() getMaxSavedParseExceptions(), getNumPersistThreads(), getMaxColumnsToMerge(), - isReleaseLocksOnHandoff() + usePerpetuallyRunningTasks ); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 9954de88c26f..c48f09b2c983 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -684,7 +684,7 @@ public void testIncrementalHandOff() throws Exception Assert.assertTrue(checkpoint1.getPartitionSequenceNumberMap().equals(currentOffsets) || checkpoint2.getPartitionSequenceNumberMap() .equals(currentOffsets)); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(1, checkpointRequestsHash.size()); @@ -787,7 +787,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); while (task.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); @@ -806,7 +806,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), nextOffsets); - task.getRunner().setEndOffsets(nextOffsets, false); + task.getRunner().setEndOffsets(nextOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -915,7 +915,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception } final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); Assert.assertEquals(1, checkpointRequestsHash.size()); @@ -994,7 +994,7 @@ public void testCheckpointResetWithSameEndOffsets() throws Exception } long currentNextCheckpointTime = task.getRunner().getNextCheckpointTime(); final Map nextEndOffsets = task.getRunner().getLastSequenceMetadata().getStartOffsets(); - task.getRunner().setEndOffsets(nextEndOffsets, false); + task.getRunner().setEndOffsets(nextEndOffsets, false, true); long newNextCheckpointTime = task.getRunner().getNextCheckpointTime(); Assert.assertTrue( StringUtils.format( @@ -1089,8 +1089,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception Map currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, false); - staleReplica.getRunner().setEndOffsets(currentOffsets, false); + normalReplica.getRunner().setEndOffsets(currentOffsets, false, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, false, true); while (normalReplica.getRunner().getStatus() != Status.PAUSED) { Thread.sleep(10); @@ -1103,8 +1103,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception currentOffsets = ImmutableMap.copyOf(staleReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint2.getPartitionSequenceNumberMap(), currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, true); - staleReplica.getRunner().setEndOffsets(currentOffsets, true); + normalReplica.getRunner().setEndOffsets(currentOffsets, true, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, true, true); Assert.assertEquals(TaskState.SUCCESS, normalReplicaFuture.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, staleReplicaFuture.get().getStatusCode()); @@ -2191,7 +2191,7 @@ public void testRestoreAfterPersistingSequences() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint.getPartitionSequenceNumberMap(), currentOffsets); // Set endOffsets to persist sequences - task1.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L), false); + task1.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L), false, true); // Stop without publishing segment task1.stopGracefully(toolboxFactory.build(task1).getConfig()); @@ -2485,12 +2485,12 @@ public void testRunWithDuplicateRequest() throws Exception // first setEndOffsets request task.getRunner().pause(); - task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true); + task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true, true); Assert.assertEquals(Status.READING, task.getRunner().getStatus()); // duplicate setEndOffsets request task.getRunner().pause(); - task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true); + task.getRunner().setEndOffsets(ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 500L), true, true); Assert.assertEquals(Status.READING, task.getRunner().getStatus()); } @@ -2931,6 +2931,8 @@ private KafkaIndexTask createTask( tuningConfig, ioConfig, context, + null, + "v1", OBJECT_MAPPER ); task.setPollRetryMs(POLL_RETRY_MS); @@ -3360,6 +3362,60 @@ public void testCompletionReportMultiplePartitionStats() throws Exception Assert.assertTrue(reportData.getRecordsProcessed().values().containsAll(ImmutableSet.of(6L, 2L))); } + @Test + public void testWithNewIoConfig() throws Exception + { + final KafkaIndexTaskIOConfig originalIoConfig = new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 0L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null, + INPUT_FORMAT, + null, + Duration.standardHours(2).getStandardMinutes() + ); + + final KafkaIndexTask originalTask = createTask("testTask", originalIoConfig); + + final KafkaIndexTaskIOConfig newIoConfig = new KafkaIndexTaskIOConfig( + 1, + "sequence1", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 5L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(new KafkaTopicPartition(false, topic, 0), 10L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + false, + null, + null, + INPUT_FORMAT, + null, + Duration.standardHours(1).getStandardMinutes() + ); + + final KafkaIndexTask newTask = (KafkaIndexTask) originalTask.withNewIoConfig(newIoConfig); + + Assert.assertNotSame("New task should be a different instance", originalTask, newTask); + Assert.assertEquals("Task ID should be preserved", originalTask.getId(), newTask.getId()); + Assert.assertEquals("Supervisor ID should be preserved", originalTask.getSupervisorId(), newTask.getSupervisorId()); + Assert.assertEquals("Task resource should be preserved", originalTask.getTaskResource(), newTask.getTaskResource()); + Assert.assertEquals("Data schema should be preserved", originalTask.getDataSchema(), newTask.getDataSchema()); + Assert.assertEquals("Tuning config should be preserved", originalTask.getTuningConfig(), newTask.getTuningConfig()); + Assert.assertEquals("Context should be preserved", originalTask.getContext(), newTask.getContext()); + Assert.assertEquals("Perpetually running flag should be preserved", originalTask.isPerpetuallyRunning(), newTask.isPerpetuallyRunning()); + + final KafkaIndexTaskIOConfig actualNewIoConfig = newTask.getIOConfig(); + Assert.assertEquals("IO config should be updated", newIoConfig, actualNewIoConfig); + Assert.assertNotEquals("IO config should be different from original", originalIoConfig, actualNewIoConfig); + + Assert.assertEquals("Start sequence should be updated", 5L, (long) actualNewIoConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap().get(new KafkaTopicPartition(false, topic, 0))); + Assert.assertEquals("End sequence should be updated", 10L, (long) actualNewIoConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap().get(new KafkaTopicPartition(false, topic, 0))); + } + public static class TestKafkaInputFormat implements InputFormat { final InputFormat baseInputFormat; diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 54ae10080a7b..abab48c7ab10 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -144,7 +144,7 @@ public void testConvert() .withReportParseExceptions(true) .withMaxColumnsToMerge(5) .build(); - KafkaIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(); + KafkaIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(null); Assert.assertEquals(original.getAppendableIndexSpec(), copy.getAppendableIndexSpec()); Assert.assertEquals(1, copy.getMaxRowsInMemory()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java index ce36c0b7dc0c..c2aa56583092 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorSpecBuilder.java @@ -30,6 +30,7 @@ public class KafkaSupervisorSpecBuilder { private String id; + private Boolean usePersistentTasks; private final DataSchema.Builder dataSchema = new DataSchema.Builder(); private final KafkaIOConfigBuilder ioConfig = new KafkaIOConfigBuilder(); private final KafkaTuningConfigBuilder tuningConfig = new KafkaTuningConfigBuilder(); @@ -57,6 +58,12 @@ public KafkaSupervisorSpecBuilder withId(String id) this.id = id; return this; } + + public KafkaSupervisorSpecBuilder withUsePersistentTasks(Boolean usePersistentTasks) + { + this.usePersistentTasks = usePersistentTasks; + return this; + } /** * Builds a new {@link KafkaSupervisorSpec} with the specified parameters. @@ -89,6 +96,7 @@ private KafkaSupervisorSpec build() ioConfig.build(), null, false, + usePersistentTasks, // Jackson injected params, not needed while posting a supervisor to the Overlord null, null, null, null, null, null, null, null, null ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 3c30cc0e25c8..bab2f66b3156 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -69,6 +69,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.TaskConfigResponse; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; @@ -133,8 +134,11 @@ import java.util.Map; import java.util.Properties; import java.util.TreeMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; @@ -233,10 +237,25 @@ public void setupTest() taskRunner = createMock(TaskRunner.class); indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); taskClient = createMock(SeekableStreamIndexTaskClient.class); + topic = getTopic(); + Map dummyOffsets = Map.of( + new KafkaTopicPartition(false, topic, 0), 100L, + new KafkaTopicPartition(false, topic, 1), 200L, + new KafkaTopicPartition(false, topic, 2), 300L + ); + EasyMock.expect(taskClient.pauseAndCheckpointAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(dummyOffsets)) + .anyTimes(); + EasyMock.expect(taskClient.getTaskConfigAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(new TaskConfigResponse<>(null, "v1"))) + .anyTimes(); + EasyMock.expect(taskClient.updateConfigAsync( + EasyMock.anyString(), + EasyMock.anyObject() + )).andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.expect(taskClient.resumeAsync(EasyMock.anyString())).andReturn(Futures.immediateFuture(true)).anyTimes(); taskQueue = createMock(TaskQueue.class); - - topic = getTopic(); topicPattern = getTopicPattern(); topicPostfix++; multiTopic = false; // assign to true in test if you wish to test multi-topic @@ -342,6 +361,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, false, + null, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -351,7 +371,14 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, new SupervisorStateManagerConfig() - ); + ) + { + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of("v1"); + } + }; supervisor = new TestableKafkaSupervisor( taskStorage, @@ -396,7 +423,7 @@ public SeekableStreamIndexTaskClient build( KafkaIndexTask task = captured.getValue(); Assert.assertEquals(KafkaSupervisorTest.dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -457,6 +484,215 @@ public SeekableStreamIndexTaskClient build( autoscaler.stop(); } + @Test + public void test_noInitialState_withPersistentTasks() throws Exception + { + final int taskCountMax = 2; + final int replicas = 1; + + // Synchronization mechanism for checkpoint coordination + final CountDownLatch checkpointTriggeredLatch = new CountDownLatch(1); + final AtomicReference checkpointTaskId = new AtomicReference<>(); + + KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( + null, + null + ) + { + @Override + public SeekableStreamIndexTaskClient build( + final String dataSource, + final TaskInfoProvider taskInfoProvider, + final SeekableStreamSupervisorTuningConfig tuningConfig, + final ScheduledExecutorService connectExec + ) + { + Assert.assertEquals(TEST_HTTP_TIMEOUT.toStandardDuration(), tuningConfig.getHttpTimeout()); + Assert.assertEquals(TEST_CHAT_RETRIES, (long) tuningConfig.getChatRetries()); + return taskClient; + } + }; + + HashMap autoScalerConfig = new HashMap<>(); + autoScalerConfig.put("enableTaskAutoScaler", true); + autoScalerConfig.put("lagCollectionIntervalMillis", 500); + autoScalerConfig.put("lagCollectionRangeMillis", 500); + autoScalerConfig.put("scaleOutThreshold", 0); + autoScalerConfig.put("triggerScaleOutFractionThreshold", 0.0); + autoScalerConfig.put("scaleInThreshold", 1000000); + autoScalerConfig.put("triggerScaleInFractionThreshold", 0.8); + autoScalerConfig.put("scaleActionStartDelayMillis", 0); + autoScalerConfig.put("scaleActionPeriodMillis", 250); + autoScalerConfig.put("taskCountMax", taskCountMax); + autoScalerConfig.put("taskCountMin", 1); + autoScalerConfig.put("scaleInStep", 1); + autoScalerConfig.put("scaleOutStep", 2); + autoScalerConfig.put("minTriggerScaleActionFrequencyMillis", 1200000); + + final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); + consumerProperties.put("myCustomKey", "myCustomValue"); + consumerProperties.put("bootstrap.servers", kafkaHost); + + KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaIOConfigBuilder() + .withTopic(topic) + .withInputFormat(INPUT_FORMAT) + .withReplicas(replicas) + .withTaskCount(1) + .withConsumerProperties(consumerProperties) + .withAutoScalerConfig(OBJECT_MAPPER.convertValue(autoScalerConfig, LagBasedAutoScalerConfig.class)) + .withUseEarliestSequenceNumber(true) + .build(); + + final KafkaSupervisorTuningConfig tuningConfigOri = new KafkaTuningConfigBuilder() + .withIntermediatePersistPeriod(Period.years(1)) + .withResetOffsetAutomatically(false) + .withWorkerThreads(numThreads) + .withShutdownTimeout(TEST_SHUTDOWN_TIMEOUT) + .withMaxRowsInMemory(1000) + .withMaxRowsPerSegment(50000) + .withReportParseExceptions(false) + .withChatHandlerNumRetries(TEST_CHAT_RETRIES) + .withChatHandlerTimeout(TEST_HTTP_TIMEOUT.toStandardDuration()) + .build(); + EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(kafkaSupervisorIOConfig).anyTimes(); + EasyMock.expect(ingestionSchema.getDataSchema()).andReturn(dataSchema).anyTimes(); + EasyMock.expect(ingestionSchema.getTuningConfig()).andReturn(tuningConfigOri).anyTimes(); + EasyMock.replay(ingestionSchema); + + SeekableStreamSupervisorSpec testableSupervisorSpec = new KafkaSupervisorSpec( + null, + ingestionSchema, + dataSchema, + tuningConfigOri, + kafkaSupervisorIOConfig, + null, + false, + true, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + new SupervisorStateManagerConfig() + ) + { + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of("v1"); + } + }; + + supervisor = new TestableKafkaSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + OBJECT_MAPPER, + (KafkaSupervisorSpec) testableSupervisorSpec, + rowIngestionMetersFactory + ); + + SupervisorTaskAutoScaler autoscaler = testableSupervisorSpec.createAutoscaler(supervisor); + + addSomeEvents(1); + + Capture captured = Capture.newInstance(); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); + EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( + new KafkaDataSourceMetadata( + null + ) + ).anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).anyTimes(); + taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); + replayAll(); + + supervisor.start(); + int taskCountBeforeScale = supervisor.getIoConfig().getTaskCount(); + Assert.assertEquals(1, taskCountBeforeScale); + autoscaler.start(); + supervisor.runInternal(); + + // After task creation, simulate task discovery by updating the mock + // to return the newly created tasks so they can be adopted + Task createdTask = captured.getValue(); + Map createdTasks = Map.of( + createdTask.getId(), createdTask + ); + EasyMock.reset(taskQueue, taskMaster, taskClient, taskStorage); + EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()).anyTimes(); + EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)) + .andReturn(createdTasks).anyTimes(); + EasyMock.expect(taskClient.getTaskConfigAsync(EasyMock.anyString())) + .andReturn(Futures.immediateFuture(new TaskConfigResponse<>(null, "v1"))) + .anyTimes(); + EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); + EasyMock.expect(taskStorage.getStatus(createdTask.getId())) + .andReturn(Optional.of(TaskStatus.running(createdTask.getId()))).anyTimes(); + EasyMock.expect(taskStorage.getTask(createdTask.getId())) + .andReturn(Optional.of(createdTask)).anyTimes(); + EasyMock.expect(taskClient.getStatusAsync(createdTask.getId())) + .andReturn(Futures.immediateFuture(Status.NOT_STARTED)).anyTimes(); + EasyMock.expect(taskClient.getStartTimeAsync(createdTask.getId())) + .andReturn(Futures.immediateFuture(DateTimes.nowUtc())).anyTimes(); + EasyMock.expect(taskClient.getCheckpointsAsync(EasyMock.eq(createdTask.getId()), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(new TreeMap<>())).anyTimes(); + EasyMock.expect(taskClient.pauseAndCheckpointAsync(EasyMock.eq(createdTask.getId()))) + .andAnswer(() -> { + // Signal that checkpoint has been triggered by autoscaler and capture task ID + checkpointTaskId.set(createdTask.getId()); + checkpointTriggeredLatch.countDown(); + return Futures.immediateFuture(singlePartitionMap(topic, 0, 0L)); + }).anyTimes(); + EasyMock.expect(taskClient.pauseAsync(EasyMock.eq(createdTask.getId()))) + .andReturn(Futures.immediateFuture(singlePartitionMap(topic, 0, 0L))).anyTimes(); + EasyMock.expect(taskClient.setEndOffsetsAsync(EasyMock.eq(createdTask.getId()), EasyMock.anyObject(), EasyMock.anyBoolean())) + .andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.expect(taskClient.updateConfigAsync(EasyMock.eq(createdTask.getId()), EasyMock.anyObject())) + .andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.expect(taskClient.resumeAsync(EasyMock.eq(createdTask.getId()))) + .andReturn(Futures.immediateFuture(true)).anyTimes(); + EasyMock.replay(taskQueue, taskMaster, taskClient, taskStorage); + + // Run again to discover and adopt the created tasks into activelyReadingTaskGroups + supervisor.runInternal(); + + // Wait for autoscaler to trigger pauseAndCheckpointAsync before proceeding with checkpoint + boolean checkpointTriggered = checkpointTriggeredLatch.await(10, TimeUnit.SECONDS); + Assert.assertTrue("Autoscaler should have triggered pauseAndCheckpointAsync within 10 seconds", checkpointTriggered); + + // Now simulate checkpoint handling - task runners hit checkpoint endpoints on overlord + // This is required for perpetually running tasks to complete the autoscaling process + String taskId = checkpointTaskId.get(); + Assert.assertNotNull("Task ID should be captured when pauseAndCheckpointAsync was called", taskId); + + Map checkpointOffsets = singlePartitionMap(topic, 0, 100L, 1, 200L, 2, 300L); + supervisor.checkpoint( + 0, // taskGroupId + new KafkaDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>(topic, checkpointOffsets, ImmutableSet.of()) + ) + ); + + // Allow some time for checkpoint processing and autoscaler to complete + Thread.sleep(2000); + verifyAll(); + + int taskCountAfterScale = supervisor.getIoConfig().getTaskCount(); + Assert.assertEquals(2, taskCountAfterScale); + + autoscaler.reset(); + autoscaler.stop(); + } + @Test public void testGetTaskRunnerType() throws JsonProcessingException { @@ -535,7 +771,7 @@ public void testNoInitialState() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -1903,7 +2139,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception for (Task task : captured.getValues()) { KafkaIndexTask kafkaIndexTask = (KafkaIndexTask) task; Assert.assertEquals(dataSchema, kafkaIndexTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), kafkaIndexTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), kafkaIndexTask.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = kafkaIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -2013,7 +2249,7 @@ public void testDiscoverExistingPublishingTask() throws Exception KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -2147,7 +2383,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KafkaIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KafkaIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals(kafkaHost, capturedTaskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -4295,7 +4531,7 @@ public void testFailedInitializationAndRecovery() throws Exception KafkaIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KafkaIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals(kafkaHost, taskConfig.getConsumerProperties().get("bootstrap.servers")); @@ -4737,11 +4973,11 @@ public void testIsTaskCurrent() replayAll(); - Assert.assertTrue(supervisor.isTaskCurrent(42, "id0", taskMap)); - Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap)); + Assert.assertTrue(supervisor.isTaskCurrent(42, "id0", taskMap, "")); + Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap, "")); verifyAll(); } @@ -5221,6 +5457,37 @@ private TestableKafkaSupervisor getTestableSupervisor( String kafkaHost, IdleConfig idleConfig ) + { + return getTestableSupervisor( + id, + replicas, + taskCount, + useEarliestOffset, + resetOffsetAutomatically, + duration, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + suspended, + kafkaHost, + idleConfig, + null + ); + } + + private TestableKafkaSupervisor getTestableSupervisor( + @Nullable String id, + int replicas, + int taskCount, + boolean useEarliestOffset, + boolean resetOffsetAutomatically, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + boolean suspended, + String kafkaHost, + IdleConfig idleConfig, + Boolean usePersistentTasks + ) { final Map consumerProperties = KafkaConsumerConfigs.getConsumerProperties(); consumerProperties.put("myCustomKey", "myCustomValue"); @@ -5287,6 +5554,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, + usePersistentTasks, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -5296,7 +5564,22 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, supervisorConfig - ), + ) + { + private String version = "v1"; + + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of(version); + } + + @Override + public void setVersion(String newVersion) + { + this.version = newVersion; + } + }, rowIngestionMetersFactory ); } @@ -5378,6 +5661,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, + null, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -5387,7 +5671,22 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, supervisorConfig - ), + ) + { + private String version = "v1"; + + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of(version); + } + + @Override + public void setVersion(String newVersion) + { + this.version = newVersion; + } + }, rowIngestionMetersFactory, isTaskCurrentReturn ); @@ -5470,6 +5769,7 @@ public SeekableStreamIndexTaskClient build( kafkaSupervisorIOConfig, null, suspended, + null, taskStorage, taskMaster, indexerMetadataStorageCoordinator, @@ -5479,7 +5779,22 @@ public SeekableStreamIndexTaskClient build( new DruidMonitorSchedulerConfig(), rowIngestionMetersFactory, supervisorConfig - ), + ) + { + private String version = "v1"; + + @Override + public java.util.Optional getVersion() + { + return java.util.Optional.of(version); + } + + @Override + public void setVersion(String newVersion) + { + this.version = newVersion; + } + }, rowIngestionMetersFactory ); } @@ -5547,7 +5862,7 @@ private KafkaIndexTask createKafkaIndexTask( minimumMessageTime, maximumMessageTime, schema, - tuningConfig.convertToTaskTuningConfig() + tuningConfig.convertToTaskTuningConfig(null) ); } @@ -5583,6 +5898,8 @@ private KafkaIndexTask createKafkaIndexTask( Duration.standardHours(2).getStandardMinutes() ), Collections.emptyMap(), + null, + "v1", OBJECT_MAPPER ); } @@ -5742,7 +6059,6 @@ public String generateSequenceName( final int groupId = getTaskGroupIdForPartition(startPartitions.keySet().iterator().next()); return StringUtils.format("sequenceName-%d", groupId); } - private SeekableStreamSupervisorStateManager getStateManager() { return stateManager; @@ -5777,7 +6093,7 @@ public TestableKafkaSupervisorWithCustomIsTaskCurrent( } @Override - public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap) + public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap, String version) { return isTaskCurrentReturn; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java new file mode 100644 index 000000000000..bc5ce28bb892 --- /dev/null +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/LagBasedAutoScalerConfigBuilder.java @@ -0,0 +1,174 @@ +/* + * 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.kafka.supervisor; + +import org.apache.druid.indexing.overlord.supervisor.autoscaler.AggregateFunction; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.LagBasedAutoScalerConfig; + +import javax.annotation.Nullable; + +/** + * Builder for {@link LagBasedAutoScalerConfig} used in tests. + */ +public class LagBasedAutoScalerConfigBuilder +{ + private long lagCollectionIntervalMillis; + private long lagCollectionRangeMillis; + private long scaleActionStartDelayMillis; + private long scaleActionPeriodMillis; + private long scaleOutThreshold; + private long scaleInThreshold; + private double triggerScaleOutFractionThreshold; + private double triggerScaleInFractionThreshold; + private int taskCountMax; + private int taskCountMin; + private Integer taskCountStart; + private int scaleInStep; + private int scaleOutStep; + private boolean enableTaskAutoScaler; + private long minTriggerScaleActionFrequencyMillis; + private AggregateFunction lagAggregate; + private double stopTaskCountRatio; + + public LagBasedAutoScalerConfigBuilder withLagCollectionIntervalMillis(long lagCollectionIntervalMillis) + { + this.lagCollectionIntervalMillis = lagCollectionIntervalMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withLagCollectionRangeMillis(long lagCollectionRangeMillis) + { + this.lagCollectionRangeMillis = lagCollectionRangeMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleActionStartDelayMillis(long scaleActionStartDelayMillis) + { + this.scaleActionStartDelayMillis = scaleActionStartDelayMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleActionPeriodMillis(long scaleActionPeriodMillis) + { + this.scaleActionPeriodMillis = scaleActionPeriodMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleOutThreshold(long scaleOutThreshold) + { + this.scaleOutThreshold = scaleOutThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleInThreshold(long scaleInThreshold) + { + this.scaleInThreshold = scaleInThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTriggerScaleOutFractionThreshold(double triggerScaleOutFractionThreshold) + { + this.triggerScaleOutFractionThreshold = triggerScaleOutFractionThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTriggerScaleInFractionThreshold(double triggerScaleInFractionThreshold) + { + this.triggerScaleInFractionThreshold = triggerScaleInFractionThreshold; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTaskCountMax(int taskCountMax) + { + this.taskCountMax = taskCountMax; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTaskCountMin(int taskCountMin) + { + this.taskCountMin = taskCountMin; + return this; + } + + public LagBasedAutoScalerConfigBuilder withTaskCountStart(@Nullable Integer taskCountStart) + { + this.taskCountStart = taskCountStart; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleInStep(int scaleInStep) + { + this.scaleInStep = scaleInStep; + return this; + } + + public LagBasedAutoScalerConfigBuilder withScaleOutStep(int scaleOutStep) + { + this.scaleOutStep = scaleOutStep; + return this; + } + + public LagBasedAutoScalerConfigBuilder withEnableTaskAutoScaler(boolean enableTaskAutoScaler) + { + this.enableTaskAutoScaler = enableTaskAutoScaler; + return this; + } + + public LagBasedAutoScalerConfigBuilder withMinTriggerScaleActionFrequencyMillis(long minTriggerScaleActionFrequencyMillis) + { + this.minTriggerScaleActionFrequencyMillis = minTriggerScaleActionFrequencyMillis; + return this; + } + + public LagBasedAutoScalerConfigBuilder withLagAggregate(AggregateFunction lagAggregate) + { + this.lagAggregate = lagAggregate; + return this; + } + + public LagBasedAutoScalerConfigBuilder withStopTaskCountRatio(double stopTaskCountRatio) + { + this.stopTaskCountRatio = stopTaskCountRatio; + return this; + } + + public LagBasedAutoScalerConfig build() + { + return new LagBasedAutoScalerConfig( + lagCollectionIntervalMillis, + lagCollectionRangeMillis, + scaleActionStartDelayMillis, + scaleActionPeriodMillis, + scaleOutThreshold, + scaleInThreshold, + triggerScaleOutFractionThreshold, + triggerScaleInFractionThreshold, + taskCountMax, + taskCountStart, + taskCountMin, + scaleInStep, + scaleOutStep, + enableTaskAutoScaler, + minTriggerScaleActionFrequencyMillis, + lagAggregate, + stopTaskCountRatio + ); + } +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index a09cfcfa5baf..3f9a229d6ce6 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.DataSchema; @@ -83,7 +84,9 @@ public KinesisIndexTask( tuningConfig, ioConfig, context, - getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE) + getFormattedGroupId(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), TYPE), + null, + null ); this.useListShards = useListShards; this.awsCredentialsConfig = awsCredentialsConfig; @@ -153,6 +156,22 @@ protected KinesisRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox) ); } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return new KinesisIndexTask( + getId(), + getSupervisorId(), + getTaskResource(), + getDataSchema(), + getTuningConfig(), + (KinesisIndexTaskIOConfig) newIoConfig, + getContext(), + useListShards, + awsCredentialsConfig + ); + } + @Override @JsonProperty public KinesisIndexTaskTuningConfig getTuningConfig() diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 884476181fa6..f491db05b412 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -149,6 +149,17 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( ); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ) + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override protected List> createIndexTasks( int replicas, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java index 8e6615716809..dea6fa3dbfa4 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorSpec.java @@ -94,7 +94,8 @@ public KinesisSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig + supervisorStateManagerConfig, + null ); this.awsCredentialsConfig = awsCredentialsConfig; } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 1a11f8d658b7..0a023caf9ed7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -252,7 +252,7 @@ public String toString() } @Override - public KinesisIndexTaskTuningConfig convertToTaskTuningConfig() + public KinesisIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new KinesisIndexTaskTuningConfig( getAppendableIndexSpec(), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 98dfea4333cc..fd57a9de3016 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -612,7 +612,7 @@ public void testIncrementalHandOff() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpoint1.getPartitionSequenceNumberMap(), currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -696,7 +696,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets1, currentOffsets); - task.getRunner().setEndOffsets(currentOffsets, false); + task.getRunner().setEndOffsets(currentOffsets, false, true); waitUntil(task, this::isTaskPaused); @@ -704,7 +704,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(checkpointOffsets2, nextOffsets); - task.getRunner().setEndOffsets(nextOffsets, false); + task.getRunner().setEndOffsets(nextOffsets, false, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -1753,7 +1753,7 @@ public void testRestoreAfterPersistingSequences() throws Exception final Map currentOffsets = ImmutableMap.copyOf(task1.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets1, currentOffsets); - task1.getRunner().setEndOffsets(currentOffsets, false); + task1.getRunner().setEndOffsets(currentOffsets, false, true); // Stop without publishing segment task1.stopGracefully(toolboxFactory.build(task1).getConfig()); @@ -1874,7 +1874,7 @@ public void testRunWithPauseAndResume() throws Exception replayAll(); - task.getRunner().setEndOffsets(currentOffsets, true); + task.getRunner().setEndOffsets(currentOffsets, true, true); Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); @@ -2033,8 +2033,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception Map currentOffsets = ImmutableMap.copyOf(normalReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets1, currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, false); - staleReplica.getRunner().setEndOffsets(currentOffsets, false); + normalReplica.getRunner().setEndOffsets(currentOffsets, false, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, false, true); waitUntil(normalReplica, this::isTaskPaused); waitUntil(staleReplica, this::isTaskPaused); @@ -2044,8 +2044,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception currentOffsets = ImmutableMap.copyOf(staleReplica.getRunner().getCurrentOffsets()); Assert.assertEquals(checkpointOffsets2, currentOffsets); - normalReplica.getRunner().setEndOffsets(currentOffsets, true); - staleReplica.getRunner().setEndOffsets(currentOffsets, true); + normalReplica.getRunner().setEndOffsets(currentOffsets, true, true); + staleReplica.getRunner().setEndOffsets(currentOffsets, true, true); Assert.assertEquals(TaskState.SUCCESS, normalReplicaFuture.get().getStatusCode()); Assert.assertEquals(TaskState.SUCCESS, staleReplicaFuture.get().getStatusCode()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 375e26e2ed28..82e1a117aa9e 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -331,7 +331,7 @@ public void testConvert() null, null ); - KinesisIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(); + KinesisIndexTaskTuningConfig copy = original.convertToTaskTuningConfig(null); Assert.assertEquals(original.getAppendableIndexSpec(), copy.getAppendableIndexSpec()); Assert.assertEquals(1, copy.getMaxRowsInMemory()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index d24c309e7c89..9347dc6d7694 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -263,7 +263,7 @@ public void testNoInitialState() throws Exception KinesisIndexTask task = captured.getValue(); Assert.assertEquals(dataSchema, task.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), task.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), task.getTuningConfig()); KinesisIndexTaskIOConfig taskConfig = task.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -1593,7 +1593,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception for (Task task : secondTasks.getValues()) { KinesisIndexTask kinesisIndexTask = (KinesisIndexTask) task; Assert.assertEquals(dataSchema, kinesisIndexTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), kinesisIndexTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), kinesisIndexTask.getTuningConfig()); KinesisIndexTaskIOConfig taskConfig = kinesisIndexTask.getIOConfig(); Assert.assertEquals("sequenceName-0", taskConfig.getBaseSequenceName()); @@ -1746,7 +1746,7 @@ public void testDiscoverExistingPublishingTask() throws Exception KinesisIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); @@ -1896,7 +1896,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() KinesisIndexTask capturedTask = captured.getValue(); Assert.assertEquals(dataSchema, capturedTask.getDataSchema()); - Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(), capturedTask.getTuningConfig()); + Assert.assertEquals(tuningConfig.convertToTaskTuningConfig(null), capturedTask.getTuningConfig()); KinesisIndexTaskIOConfig capturedTaskConfig = capturedTask.getIOConfig(); Assert.assertEquals("awsEndpoint", capturedTaskConfig.getEndpoint()); @@ -4094,10 +4094,10 @@ public void testIsTaskCurrent() replayAll(); - Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap)); - Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap)); + Assert.assertTrue(supervisor.isTaskCurrent(42, "id1", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id2", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id3", taskMap, "")); + Assert.assertFalse(supervisor.isTaskCurrent(42, "id4", taskMap, "")); verifyAll(); } @@ -5603,7 +5603,7 @@ private KinesisIndexTask createKinesisIndexTask( minimumMessageTime, maximumMessageTime, dataSchema, - (KinesisIndexTaskTuningConfig) tuningConfig.convertToTaskTuningConfig() + (KinesisIndexTaskTuningConfig) tuningConfig.convertToTaskTuningConfig(null) ); } @@ -5772,7 +5772,7 @@ private class TestableKinesisSupervisorWithCustomIsTaskCurrent extends TestableK } @Override - public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap) + public boolean isTaskCurrent(int taskGroupId, String taskId, Map taskMap, String version) { return isTaskCurrentReturn; } diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java index 90d9f3ac420e..c87d436114a9 100644 --- a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyLagAggregator.java @@ -25,29 +25,50 @@ import org.apache.druid.indexing.seekablestream.supervisor.LagAggregator; import org.apache.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** * Implementation of {@link LagAggregator} that supports the following: *
    *
  • Specify a {@code multiplier} to amplify the lag observed by the Overlord * for a given supervisor.
  • + *
  • Support externally controllable lag injection for testing auto-scaling scenarios.
  • *
*/ public class FaultyLagAggregator implements LagAggregator { private static final Logger log = new Logger(FaultyLagAggregator.class); + // Global registry for external control from tests + private static final ConcurrentHashMap INJECTED_LAG_REGISTRY = new ConcurrentHashMap<>(); + private final int lagMultiplier; + private final String clientId; private final LagAggregator delegate = LagAggregator.DEFAULT; + public FaultyLagAggregator(int lagMultiplier) + { + this.lagMultiplier = lagMultiplier; + this.clientId = null; + log.info("Multiplying lags by factor[%d].", lagMultiplier); + } + @JsonCreator public FaultyLagAggregator( - @JsonProperty("lagMultiplier") int lagMultiplier + @JsonProperty("lagMultiplier") int lagMultiplier, + @JsonProperty("clientId") @Nullable String clientId ) { this.lagMultiplier = lagMultiplier; - log.info("Multiplying lags by factor[%d].", lagMultiplier); + this.clientId = clientId; + + if (clientId != null) { + log.info("Controllable faulty lag aggregator with extenral ID[%s]", clientId); + } else { + log.info("Multiplying lags by factor[%d].", lagMultiplier); + } } @JsonProperty @@ -56,8 +77,36 @@ public int getLagMultiplier() return lagMultiplier; } + @JsonProperty + @Nullable + public String getClientId() + { + return clientId; + } + @Override public LagStats aggregate(Map partitionLags) + { + if (clientId != null && INJECTED_LAG_REGISTRY.containsKey(clientId)) { + return aggregateWithInjectedLag(partitionLags); + } else { + return aggregateWithMultiplier(partitionLags); + } + } + + private LagStats aggregateWithInjectedLag(Map partitionLags) + { + long injectedLag = INJECTED_LAG_REGISTRY.get(clientId); + + log.debug("Using injected lag[%d] for controller[%s]", injectedLag, clientId); + + // Return fixed lag values regardless of actual partition lags + long totalLag = partitionLags.isEmpty() ? injectedLag : injectedLag * partitionLags.size(); + + return new LagStats(injectedLag, totalLag, injectedLag); + } + + private LagStats aggregateWithMultiplier(Map partitionLags) { LagStats originalAggregate = delegate.aggregate(partitionLags); return new LagStats( @@ -66,4 +115,16 @@ public LagStats aggregate(Map partition originalAggregate.getAvgLag() * getLagMultiplier() ); } + + public static void injectLag(String controllerId, long lag) + { + log.info("Injecting lag[%d] for controller[%s]", lag, controllerId); + INJECTED_LAG_REGISTRY.put(controllerId, lag); + } + + public static void clearAllInjectedLag() + { + log.info("Clearing all injected lag"); + INJECTED_LAG_REGISTRY.clear(); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 24f74b8da8b5..43c0386c34ca 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -71,6 +71,8 @@ public abstract class SeekableStreamIndexTask ioConfig, @Nullable final Map context, - @Nullable final String groupId + @Nullable final String groupId, + @Nullable final Boolean isPerpetuallyRunning, + @Nullable final String supervisorSpecVersion ) { super( @@ -106,6 +110,8 @@ public SeekableStreamIndexTask( : LockGranularity.SEGMENT; this.lockTypeToUse = TaskLocks.determineLockTypeForAppend(getContext()); this.supervisorId = Preconditions.checkNotNull(Configs.valueOrDefault(supervisorId, dataSchema.getDataSource()), "supervisorId"); + this.isPerpetuallyRunning = Configs.valueOrDefault(isPerpetuallyRunning, false); + this.supervisorSpecVersion = Configs.valueOrDefault(supervisorSpecVersion, ""); } protected static String getFormattedGroupId(String supervisorId, String type) @@ -153,6 +159,18 @@ public SeekableStreamIndexTaskIOConfig getI return ioConfig; } + @JsonProperty("isPerpetuallyRunning") + public boolean isPerpetuallyRunning() + { + return isPerpetuallyRunning; + } + + @JsonProperty("supervisorSpecVersion") + public String getSupervisorSpecVersion() + { + return supervisorSpecVersion; + } + @Override public TaskStatus runTask(final TaskToolbox toolbox) { @@ -309,4 +327,8 @@ public Appenderator getAppenderator() { return runnerSupplier.get(); } + + public abstract SeekableStreamIndexTask withNewIoConfig( + SeekableStreamIndexTaskIOConfig newIoConfig + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java index 484ad15f7cd6..e35ec1b61fa4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClient.java @@ -91,6 +91,8 @@ ListenableFuture>> get */ ListenableFuture> pauseAsync(String id); + ListenableFuture> pauseAndCheckpointAsync(String id); + /** * Set end offsets for a task. Retries on failure. * @@ -168,6 +170,20 @@ ListenableFuture registerNewVersionOfPendingSegmentAsync( PendingSegmentRecord pendingSegmentRecord ); + /** + * Update the configuration of a running task. + * + * Task-side is {@link SeekableStreamIndexTaskRunner#updateConfig}. + */ + ListenableFuture updateConfigAsync(String taskId, TaskConfigUpdateRequest updateRequest); + + /** + * Gets the current configuration of a running task. + * + * Task-side is {@link SeekableStreamIndexTaskRunner#getConfigHTTP}. + */ + ListenableFuture> getTaskConfigAsync(String taskId); + Class getPartitionType(); Class getSequenceType(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java index 6af2735e995b..693f7cf585bf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImpl.java @@ -210,6 +210,45 @@ public ListenableFuture registerNewVersionOfPendingSegmentAsync( .go(); } + @Override + public ListenableFuture updateConfigAsync( + String taskId, + TaskConfigUpdateRequest updateRequest + ) + { + final RequestBuilder requestBuilder = new RequestBuilder(HttpMethod.POST, "/config") + .jsonContent(jsonMapper, updateRequest); + return makeRequest(taskId, requestBuilder) + .handler(IgnoreHttpResponseHandler.INSTANCE) + .onSuccess(r -> { + log.info("Successfully updated config for task [%s]", taskId); + return true; + }) + .onHttpError(e -> { + log.warn("Task [%s] config update failed due to http request failure [%s].", taskId, e.getMessage()); + return Either.value(false); + }) + .onNotAvailable(e -> { + log.warn("Task [%s] config update failed because task is not available.", taskId); + return Either.value(false); + }) + .onClosed(e -> { + log.warn("Task [%s] config update failed because task is no longer running.", taskId); + return Either.value(false); + }) + .go(); + } + + @Override + public ListenableFuture> getTaskConfigAsync(String taskId) + { + return makeRequest(taskId, new RequestBuilder(HttpMethod.GET, "/config")) + .handler(new BytesFullResponseHandler()) + .onSuccess(r -> (TaskConfigResponse) JacksonUtils.readValue(jsonMapper, r.getContent(), TaskConfigResponse.class)) + .onNotAvailable(e -> Either.value(null)) + .go(); + } + @Override public ListenableFuture setEndOffsetsAsync( final String id, @@ -301,6 +340,50 @@ public ListenableFuture> pauseAsync(Str ); } + @Override + public ListenableFuture> pauseAndCheckpointAsync(String id) + { + final ListenableFuture> pauseFuture = + makeRequest(id, new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint")) + .handler(new BytesFullResponseHandler()) + .onSuccess(r -> { + if (r.getStatus().equals(HttpResponseStatus.OK)) { + log.info("Task [%s] paused successfully & Checkpoint requested successffully", id); + return deserializeOffsetsMap(r.getContent()); + } else if (r.getStatus().equals(HttpResponseStatus.ACCEPTED)) { + return null; + } else { + throw new ISE( + "Pause & Checkpoint request for task [%s] failed with response [%s]", + id, + r.getStatus() + ); + } + }) + .onNotAvailable(e -> Either.value(Collections.emptyMap())) + .go(); + + return FutureUtils.transformAsync( + pauseFuture, + result -> { + if (result != null) { + return Futures.immediateFuture(result); + } else { + return getOffsetsWhenPaused( + id, + new RetryPolicyFactory( + new RetryPolicyConfig() + .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS)) + .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS)) + .setMaxRetryCount(httpRetries) + ).makeRetryPolicy() + ); + } + } + ); + } + + @Override public ListenableFuture> getMovingAveragesAsync(String id) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index c6e4a9ff2356..0a6d230e48c4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -78,7 +78,9 @@ 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.ServiceMetricEvent; import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -156,12 +158,11 @@ public enum Status PAUSED, PUBLISHING } - private static final EmittingLogger log = new EmittingLogger(SeekableStreamIndexTaskRunner.class); static final String METADATA_NEXT_PARTITIONS = "nextPartitions"; static final String METADATA_PUBLISH_PARTITIONS = "publishPartitions"; - private final Map endOffsets; + private Map endOffsets; // lastReadOffsets are the last offsets that were read and processed. private final Map lastReadOffsets = new HashMap<>(); @@ -206,7 +207,10 @@ public enum Status protected final Condition isAwaitingRetry = pollRetryLock.newCondition(); private final SeekableStreamIndexTask task; - private final SeekableStreamIndexTaskIOConfig ioConfig; + private volatile Set> assignment; + private volatile RecordSupplier recordSupplier; + private SeekableStreamIndexTaskIOConfig ioConfig; + private volatile String supervisorSpecVersion; private final SeekableStreamIndexTaskTuningConfig tuningConfig; private final InputRowSchema inputRowSchema; @Nullable @@ -251,6 +255,8 @@ public enum Status private volatile DateTime minMessageTime; private volatile DateTime maxMessageTime; private final ScheduledExecutorService rejectionPeriodUpdaterExec; + private final AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); + public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, @@ -270,6 +276,7 @@ public SeekableStreamIndexTaskRunner( this.sequences = new CopyOnWriteArrayList<>(); this.ingestionState = IngestionState.NOT_STARTED; this.lockGranularityToUse = lockGranularityToUse; + this.supervisorSpecVersion = task.getSupervisorSpecVersion(); minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); @@ -452,9 +459,10 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception //milliseconds waited for created segments to be handed off long handoffWaitMs = 0L; - + log.info("Task perpetually running: %s", task.isPerpetuallyRunning()); try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier(toolbox)) { + this.recordSupplier = recordSupplier; if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) { toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); @@ -599,7 +607,7 @@ public void run() // restart publishing of sequences (if any) maybePersistAndPublishSequences(committerSupplier); - Set> assignment = assignPartitions(recordSupplier); + assignment = assignPartitions(recordSupplier); possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); seekToStartingSequence(recordSupplier, assignment); @@ -607,7 +615,7 @@ public void run() // Main loop. // Could eventually support leader/follower mode (for keeping replicas more in sync) - boolean stillReading = !assignment.isEmpty(); + boolean stillReading = isStillReading(); status = Status.READING; Throwable caughtExceptionInner = null; @@ -620,7 +628,7 @@ public void run() assignment = assignPartitions(recordSupplier); possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); - if (assignment.isEmpty()) { + if (assignment.isEmpty() && !task.isPerpetuallyRunning()) { log.debug("All partitions have been fully read."); publishOnStop.set(true); stopRequested.set(true); @@ -652,7 +660,7 @@ public void run() ); // note: getRecords() also updates assignment - stillReading = !assignment.isEmpty(); + stillReading = isStillReading(); SequenceMetadata sequenceToCheckpoint = null; AppenderatorDriverAddResult pushTriggeringAddResult = null; @@ -757,7 +765,7 @@ public void onFailure(Throwable t) if (!moreToReadAfterThisRecord && assignment.remove(record.getStreamPartition())) { log.info("Finished reading stream[%s], partition[%s].", record.getStream(), record.getPartitionId()); recordSupplier.assign(assignment); - stillReading = !assignment.isEmpty(); + stillReading = isStillReading(); } } @@ -909,7 +917,6 @@ public void onFailure(Throwable t) String.join(", ", Lists.transform(handedOff.getSegments(), DataSegment::toString)) ); } - appenderator.close(); } catch (InterruptedException | RejectedExecutionException e) { @@ -981,6 +988,11 @@ public void onFailure(Throwable t) return TaskStatus.success(task.getId()); } + private boolean isStillReading() + { + return !assignment.isEmpty() || task.isPerpetuallyRunning(); + } + private TaskLockType getTaskLockType() { return TaskLocks.determineLockTypeForAppend(task.getContext()); @@ -1275,7 +1287,7 @@ private void addSequence(final SequenceMetadata getEndOffsets() return endOffsets; } + @GET + @Path("/config") + @Produces(MediaType.APPLICATION_JSON) + public TaskConfigResponse getConfigHTTP(@Context final HttpServletRequest req) + { + authorizationCheck(req); + return new TaskConfigResponse<>(ioConfig, supervisorSpecVersion); + } + @POST @Path("/offsets/end") @Consumes(MediaType.APPLICATION_JSON) @@ -1612,7 +1633,7 @@ public Response setEndOffsetsHTTP( ) throws InterruptedException { authorizationCheck(req); - return setEndOffsets(sequences, finish); + return setEndOffsets(sequences, finish, !waitForConfigUpdate.get()); } @POST @@ -1717,10 +1738,161 @@ public Response getUnparseableEvents( return Response.ok(parseExceptionHandler.getSavedParseExceptionReports()).build(); } + @POST + @Path("/config") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response updateConfig( + TaskConfigUpdateRequest request, + @Context final HttpServletRequest req + ) + { + authorizationCheck(req); + if (!waitForConfigUpdate.get()) { + return Response.status(409).entity("Task must have been paused and checkpointed before updating config.").build(); + } + try { + return updateTaskRunnerConfig(request); + } + catch (Exception e) { + log.makeAlert(e, "Failed to update task config"); + waitForConfigUpdate.set(false); + return Response.serverError().entity(e.getMessage()).build(); + } + } + + /** + * Updates the task's ioConfig, creates a new sequence from the new ioConfig, re-assigns partitions and seeks to + * the new starting offsets. If there is no partition assigned to this task due to a scale down, the task will be + * paused. + */ + private Response updateTaskRunnerConfig(TaskConfigUpdateRequest request) + throws IOException, InterruptedException + { + log.info("Attempting to update config to [%s]", request.getIoConfig()); + SeekableStreamIndexTaskIOConfig newIoConfig = request.getIoConfig(); + setIOConfig(newIoConfig); + createNewSequenceFromIoConfig(newIoConfig, request.getLastOffsets()); + supervisorSpecVersion = request.getSupervisorSpecVersion(); + + assignment = assignPartitions(recordSupplier); + if (!assignment.isEmpty()) { + possiblyResetDataSourceMetadata(toolbox, recordSupplier, assignment); + seekToStartingSequence(recordSupplier, assignment); + resume(); + } + + log.info("Config updated to [%s]", this.ioConfig); + toolbox.getEmitter().emit(ServiceMetricEvent.builder() + .setDimension(DruidMetrics.TASK_ID, task.getId()) + .setDimension(DruidMetrics.TASK_TYPE, task.getType()) + .setDimension(DruidMetrics.DATASOURCE, task.getDataSource()) + .setMetric("task/config/update/success", 1) + .build(ImmutableMap.of())); + waitForConfigUpdate.set(false); + return Response.ok().build(); + } + + private void setIOConfig( + SeekableStreamIndexTaskIOConfig ioConfig + ) + { + this.ioConfig = ioConfig; + this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); + this.minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); + this.maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); + } + + /** + * Creates new sequences for the ingestion process. It currently accepts the ioConfig given by the request as the correct offsets + * and ignores the offsets it may have stored in currOffsets and endOffsets. + */ + private void createNewSequenceFromIoConfig(SeekableStreamIndexTaskIOConfig ioConfig, + Map latestCommittedOffsets + ) + throws IOException + { + Map partitionStartOffsets = ioConfig.getStartSequenceNumbers() + .getPartitionSequenceNumberMap(); + Map partitionEndSequences = ioConfig.getEndSequenceNumbers() + .getPartitionSequenceNumberMap(); + SequenceMetadata lastSequenceMetadata = getLastSequenceMetadata(); + Map offsetsForLastPartitionAssignment = latestCommittedOffsets.entrySet() + .stream() + .filter(e -> lastSequenceMetadata.startOffsets.containsKey(e.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + if (lastSequenceMetadata.startOffsets.equals(offsetsForLastPartitionAssignment)) { + // This is the case where no data has been ingested since the last successfull checkpoint for these partitions. + // In this case, we'll mark the end offsets for this sequence same as start offsets and create a new sequence. + log.info("No new data ingested across any sibling tasks for this partition sequence since last checkpoint."); + lastSequenceMetadata.setEndOffsets(lastSequenceMetadata.startOffsets); + } + + final Set exclusiveStartPartitions = computeExclusiveStartPartitionsForSequence( + partitionStartOffsets); + + final SequenceMetadata newSequence = new SequenceMetadata<>( + sequences.isEmpty() ? 0 : lastSequenceMetadata.getSequenceId() + 1, + StringUtils.format( + "%s_%d", + ioConfig.getBaseSequenceName(), + sequences.isEmpty() ? 0 : lastSequenceMetadata.getSequenceId() + 1 + ), + partitionStartOffsets, + partitionEndSequences, + false, + exclusiveStartPartitions, + getTaskLockType() + ); + log.info("Attempting adding new sequence [%s]", newSequence); + + currOffsets.clear(); + currOffsets.putAll(partitionStartOffsets); + endOffsets.clear(); + endOffsets.putAll(partitionEndSequences); + + addSequence(newSequence); + persistSequences(); + log.info( + "Created new sequence [%s] with start offsets [%s]", + newSequence.getSequenceName(), partitionStartOffsets + ); + } + + private void checkpointSequences() + { + try { + final SequenceMetadata latestSequence = getLastSequenceMetadata(); + if (!latestSequence.isCheckpointed()) { + final CheckPointDataSourceMetadataAction checkpointAction = new CheckPointDataSourceMetadataAction( + getSupervisorId(), + ioConfig.getTaskGroupId(), + null, + createDataSourceMetadata( + new SeekableStreamStartSequenceNumbers<>( + stream, + latestSequence.getStartOffsets(), + latestSequence.getExclusiveStartPartitions() + ) + ) + ); + if (!toolbox.getTaskActionClient().submit(checkpointAction)) { + throw new ISE("Checkpoint request with sequences [%s] failed, dying", currOffsets); + } + } + } + catch (Exception e) { + log.error(e, "Failed to checkpoint sequences."); + backgroundThreadException = e; + } + } + @VisibleForTesting public Response setEndOffsets( Map sequenceNumbers, - boolean finish // this field is only for internal purposes, shouldn't be usually set by users + boolean finish, // this field is only for internal purposes, shouldn't be usually set by users + boolean shouldPause ) throws InterruptedException { if (sequenceNumbers == null) { @@ -1738,6 +1910,7 @@ public Response setEndOffsets( .build(); } else { try { + // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock pauseLock.lockInterruptibly(); // Perform all sequence related checks before checking for isPaused() // and after acquiring pauseLock to correctly guard against duplicate requests @@ -1762,7 +1935,9 @@ public Response setEndOffsets( || (latestSequence.getEndOffsets().equals(sequenceNumbers) && finish)) { log.warn("Ignoring duplicate request, end offsets already set for sequences [%s]", sequenceNumbers); resetNextCheckpointTime(); - resume(); + if (shouldPause) { + resume(); + } return Response.ok(sequenceNumbers).build(); } else if (latestSequence.isCheckpointed()) { return Response.status(Response.Status.BAD_REQUEST) @@ -1795,8 +1970,10 @@ public Response setEndOffsets( resetNextCheckpointTime(); latestSequence.setEndOffsets(sequenceNumbers); - - if (finish) { + log.info("Latest sequence after setting end offsets: [%s]", latestSequence); + // if this is the final checkpoint, or if the task is paused for checkpoint completion and updateConfig is supposed to + // finish the current sequence, we just update the end offsets of the latest sequence. + if (finish || waitForConfigUpdate.get()) { log.info( "Sequence[%s] end offsets updated from [%s] to [%s].", latestSequence.getSequenceName(), @@ -1841,7 +2018,9 @@ public Response setEndOffsets( } } - resume(); + if (shouldPause) { + resume(); + } return Response.ok(sequenceNumbers).build(); } @@ -1895,6 +2074,32 @@ public Response pauseHTTP( return pause(); } + @POST + @Path("/pauseAndCheckpoint") + @Produces(MediaType.APPLICATION_JSON) + public Response pauseAndCheckpointHTTP( + @Context final HttpServletRequest req + ) throws InterruptedException + { + authorizationCheck(req); + if (!waitForConfigUpdate.compareAndSet(false, true)) { + return Response.ok().entity("Task is already paused for checkpoint completion").build(); + } + Response pauseResponse = pause(); + if (pauseResponse.getStatus() == 409) { + waitForConfigUpdate.set(false); + return pauseResponse; + } + try { + checkpointSequences(); + } + catch (Exception e) { + waitForConfigUpdate.set(false); + resume(); + } + return Response.ok().entity(getCurrentOffsets()).build(); + } + @VisibleForTesting public Response pause() throws InterruptedException { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java new file mode 100644 index 000000000000..62af177414d0 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigResponse.java @@ -0,0 +1,86 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Response object for returning the configuration of a running {@link SeekableStreamIndexTask}. + */ +public class TaskConfigResponse +{ + private final SeekableStreamIndexTaskIOConfig ioConfig; + private final String supervisorSpecVersion; + + @JsonCreator + public TaskConfigResponse( + @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig, + @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion + ) + { + this.ioConfig = ioConfig; + this.supervisorSpecVersion = supervisorSpecVersion; + } + + @JsonProperty + public SeekableStreamIndexTaskIOConfig getIoConfig() + { + return ioConfig; + } + + @JsonProperty + public String getSupervisorSpecVersion() + { + return supervisorSpecVersion; + } + + @Override + public String toString() + { + return "TaskConfigResponse{" + + "ioConfig=" + ioConfig + + ", supervisorSpecVersion='" + supervisorSpecVersion + '\'' + + '}'; + } + + @Override + public boolean equals(Object object) + { + if (object == null || getClass() != object.getClass()) { + return false; + } + TaskConfigResponse that = (TaskConfigResponse) object; + return Objects.equals(ioConfig, that.ioConfig) && Objects.equals( + supervisorSpecVersion, + that.supervisorSpecVersion + ); + } + + @Override + public int hashCode() + { + return Objects.hash(ioConfig, supervisorSpecVersion); + } + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java new file mode 100644 index 000000000000..9528135e71a7 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequest.java @@ -0,0 +1,95 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Objects; + +/** + * Request object for updating the configuration of a running {@link SeekableStreamIndexTask}. + */ +public class TaskConfigUpdateRequest +{ + private final SeekableStreamIndexTaskIOConfig ioConfig; + private final String supervisorSpecVersion; + private final Map lastOffsets; + + @JsonCreator + public TaskConfigUpdateRequest( + @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig ioConfig, + @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion, + @JsonProperty("lastOffsets") Map lastOffsets + ) + { + this.ioConfig = ioConfig; + this.supervisorSpecVersion = supervisorSpecVersion; + this.lastOffsets = lastOffsets; + } + + @JsonProperty + public String getSupervisorSpecVersion() + { + return supervisorSpecVersion; + } + + @JsonProperty + public SeekableStreamIndexTaskIOConfig getIoConfig() + { + return ioConfig; + } + + @JsonProperty + public Map getLastOffsets() + { + return lastOffsets; + } + + @Override + public boolean equals(Object object) + { + if (object == null || getClass() != object.getClass()) { + return false; + } + TaskConfigUpdateRequest that = (TaskConfigUpdateRequest) object; + return Objects.equals(ioConfig, that.ioConfig) && Objects.equals( + supervisorSpecVersion, + that.supervisorSpecVersion + ); + } + + @Override + public int hashCode() + { + return Objects.hash(ioConfig, supervisorSpecVersion); + } + + @Override + public String toString() + { + return "TaskConfigUpdateRequest{" + + "ioConfig=" + ioConfig + + ", supervisorSpecVersion='" + supervisorSpecVersion + '\'' + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java index cbc61edbe6ee..24fe811c06f1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumber.java @@ -98,6 +98,11 @@ public boolean isAvailableWithEarliest(OrderedSequenceNumber public boolean isMoreToReadBeforeReadingRecord(OrderedSequenceNumber end, boolean isEndOffsetExclusive) { + // This happens in the situations where earlier sequences had a different partition mapping and has now been updated. + // Since the end is not defined, we can't really say if there is more to read or not. + if (end.sequenceNumber == null) { + return false; + } final int compareToEnd = this.compareTo(end); return isEndOffsetExclusive ? compareToEnd < 0 : compareToEnd <= 0; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 312a6480e253..72b390cef288 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -73,6 +73,8 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.TaskConfigResponse; +import org.apache.druid.indexing.seekablestream.TaskConfigUpdateRequest; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamException; @@ -128,6 +130,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiConsumer; @@ -195,14 +198,17 @@ public class TaskGroup // this task group has completed successfully, at which point this will be destroyed and a new task group will be // created with new starting sequences. This allows us to create replacement tasks for failed tasks that process the // same sequences, even if the values in [partitionGroups] has been changed. - final ImmutableMap startingSequences; + // In perpetually-running tasks mode, the actively running task groups will be replaced with new task groups with updated starting sequences. + ImmutableMap startingSequences; // We don't include closed partitions in the starting offsets. However, we keep the full unfiltered map of // partitions, only used for generating the sequence name, to avoid ambiguity in sequence names if mulitple // task groups have nothing but closed partitions in their assignments. + final ImmutableMap unfilteredStartingSequencesForSequenceName; final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + final DateTime minimumMessageTime; final DateTime maximumMessageTime; final Set exclusiveStartSequenceNumberPartitions; @@ -212,6 +218,21 @@ public class TaskGroup boolean handoffEarly = false; // set by SupervisorManager.stopTaskGroupEarly + public int getId() + { + return groupId; + } + + public DateTime getMinimumMessageTime() + { + return minimumMessageTime; + } + + public DateTime getMaximumMessageTime() + { + return maximumMessageTime; + } + TaskGroup( int groupId, ImmutableMap startingSequences, @@ -298,6 +319,12 @@ public String toString() ", tasks=" + tasks + '}'; } + + public TaskGroup withStartingSequences(Map newStartingSequences) + { + this.startingSequences = ImmutableMap.copyOf(newStartingSequences); + return this; + } } private class TaskData @@ -520,8 +547,8 @@ public void handle() emitter.emit(event.setMetric(AUTOSCALER_REQUIRED_TASKS_METRIC, desiredTaskCount)); } - boolean allocationSuccess = changeTaskCount(desiredTaskCount); - if (allocationSuccess) { + boolean allocationSuccess = changeTaskCount(desiredTaskCount, onSuccessfulScale); + if (allocationSuccess && !spec.usePersistentTasks()) { onSuccessfulScale.run(); dynamicTriggerLastRunTime = nowTime; } @@ -548,12 +575,13 @@ public String getType() * After the taskCount is changed in SeekableStreamSupervisorIOConfig, next RunNotice will create scaled number of ingest tasks without resubmitting the supervisor. * * @param desiredActiveTaskCount desired taskCount computed from AutoScaler + * @param successfulScaleAutoScalerCallback callback to be triggered on successful scale action for AutoScaler to reset its queue. * @return Boolean flag indicating if scale action was executed or not. If true, it will wait at least 'minTriggerScaleActionFrequencyMillis' before next 'changeTaskCount'. * If false, it will do 'changeTaskCount' again after 'scaleActionPeriodMillis' millis. * @throws InterruptedException * @throws ExecutionException */ - private boolean changeTaskCount(int desiredActiveTaskCount) + private boolean changeTaskCount(int desiredActiveTaskCount, Runnable successfulScaleAutoScalerCallback) throws InterruptedException, ExecutionException { int currentActiveTaskCount; @@ -571,29 +599,306 @@ private boolean changeTaskCount(int desiredActiveTaskCount) dataSource ); final Stopwatch scaleActionStopwatch = Stopwatch.createStarted(); - gracefulShutdownInternal(); - changeTaskCountInIOConfig(desiredActiveTaskCount); - clearAllocationInfo(); - emitter.emit(ServiceMetricEvent.builder() - .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) - .setDimension(DruidMetrics.DATASOURCE, dataSource) - .setDimension(DruidMetrics.STREAM, getIoConfig().getStream()) - .setDimensionIfNotNull( - DruidMetrics.TAGS, - spec.getContextValue(DruidMetrics.TAGS) - ) - .setMetric( - AUTOSCALER_SCALING_TIME_METRIC, - scaleActionStopwatch.millisElapsed() - )); + + if (spec.usePersistentTasks()) { + return changeTaskCountForPerpetualTasks(desiredActiveTaskCount, successfulScaleAutoScalerCallback); + } else { + gracefulShutdownInternal(); + changeTaskCountInIOConfig(desiredActiveTaskCount); + clearAllocationInfo(); + } + emitAutoScalerRunMetric(scaleActionStopwatch); log.info("Changed taskCount to [%s] for supervisor[%s] for dataSource[%s].", desiredActiveTaskCount, supervisorId, dataSource); return true; } } + private void emitAutoScalerRunMetric(Stopwatch scaleActionStopwatch) + { + emitter.emit(ServiceMetricEvent.builder() + .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) + .setDimension(DruidMetrics.DATASOURCE, dataSource) + .setDimension(DruidMetrics.STREAM, getIoConfig().getStream()) + .setDimensionIfNotNull( + DruidMetrics.TAGS, + spec.getContextValue(DruidMetrics.TAGS) + ) + .setMetric( + AUTOSCALER_SCALING_TIME_METRIC, + scaleActionStopwatch.millisElapsed() + )); + } + + /** + * Handles task count changes for perpetual tasks using updateConfig instead of graceful shutdown. + * This approach pauses tasks, recalculates partition assignments, and sends config updates. + */ + private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount, + Runnable successfulScaleAutoScalerCallback + ) + throws InterruptedException, ExecutionException + { + if (!isDynamicAllocationOngoing.compareAndSet(false, true)) { + log.info("A dynamic allocation is already ongoing, skipping this request."); + return false; + } + final Stopwatch scaleActionStopwatch = Stopwatch.createStarted(); + log.info("Handling task count change for perpetual tasks from [%d] to [%d]", + activelyReadingTaskGroups.size(), desiredActiveTaskCount); + + Map offsetsFromTasks = pauseAndCheckpointAllTasks(); + + if (offsetsFromTasks.isEmpty()) { + isDynamicAllocationOngoing.set(false); + return false; + } + pendingConfigUpdateHook = () -> updateTaskConfigsAndCompleteAutoScaleEvent( + offsetsFromTasks, + scaleActionStopwatch, + desiredActiveTaskCount, + successfulScaleAutoScalerCallback + ); + return true; + } + + /** + * This function sends the config updates to all the tasks with new partition assignments and offsets. In an auto-scaling + * lifecycle, this marks the end of the scale action. + */ + private boolean updateTaskConfigsAndCompleteAutoScaleEvent( + Map offsetsFromTasks, + Stopwatch scaleActionStopwatch, + int desiredActiveTaskCount, + Runnable successfulScaleAutoScalerCallback + ) throws InterruptedException, ExecutionException + { + changeTaskCountInIOConfig(desiredActiveTaskCount); + Map> newPartitionGroups = calculateNewPartitionGroups(); + + boolean success = sendConfigUpdatesToTasks(newPartitionGroups, offsetsFromTasks); + + if (success) { + updatePartitionGroupsForPerpetualTasks(newPartitionGroups); + + log.info("Successfully updated task configurations for perpetual tasks scaling"); + } else { + log.error("Failed to update task configurations for perpetual tasks"); + } + + isDynamicAllocationOngoing.set(false); + emitAutoScalerRunMetric(scaleActionStopwatch); + + // You need to set the auto scaler specific stuff here. + successfulScaleAutoScalerCallback.run(); + dynamicTriggerLastRunTime = System.currentTimeMillis(); + return success; + } + + + private Map pauseAndCheckpointAllTasks() throws InterruptedException, ExecutionException + { + log.info("Pausing all tasks for perpetual task scaling"); + List>> pauseFutures = new ArrayList<>(); + + for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) { + for (String taskId : taskGroup.taskIds()) { + log.debug("Pausing & Checkpointing tasks [%s]", taskId); + pauseFutures.add(taskClient.pauseAndCheckpointAsync(taskId)); + } + } + + if (pauseFutures.isEmpty()) { + log.warn("No tasks found to pause for perpetual task scaling"); + return Collections.emptyMap(); + } + + Map consolidatedLatestOffsets = new HashMap<>(); + + int tasksToWaitFor = pauseFutures.size(); + checkpointsToWaitFor = tasksToWaitFor; + log.info("Broadcasting pauseAndCheckpoint to [%d] tasks", tasksToWaitFor); + + List>> latestOffsets = coalesceAndAwait(pauseFutures); + for (Either> result : latestOffsets) { + if (result.isError()) { + log.warn("Failed to pause and checkpoint task[%s]", result.error().getMessage()); + } else { + Map offsets = result.valueOrThrow(); + offsets.forEach((partition, offset) -> + consolidatedLatestOffsets.merge( + partition, + offset, + (key, existingOffset) -> { + OrderedSequenceNumber existing = makeSequenceNumber(existingOffset); + OrderedSequenceNumber incoming = makeSequenceNumber(offset); + return existing.compareTo(incoming) >= 0 ? existingOffset : offset; + } + )); + } + } + + log.info("Successfully paused & checkpoints requested from [%d] tasks", pauseFutures.size()); + return consolidatedLatestOffsets; + } + + /** + * Calculates new partition groups. + */ + private Map> calculateNewPartitionGroups() + { + log.info("Calculating new partition groups using getTaskGroupIdForPartition() logic"); + Map> newPartitionGroups = new HashMap<>(); + + List allPartitions = new ArrayList<>(partitionIds); + + if (allPartitions.isEmpty()) { + log.warn("No partitions available for assignment"); + return newPartitionGroups; + } + + for (PartitionIdType partition : allPartitions) { + int taskGroupId = getTaskGroupIdForPartition(partition); + newPartitionGroups.computeIfAbsent(taskGroupId, k -> new HashSet<>()).add(partition); + } + + log.info("Created [%d] new partition groups: %s", newPartitionGroups.size(), newPartitionGroups); + return newPartitionGroups; + } + + /** + * Sends configuration updates to tasks with new partition assignments. + * Also handles cleanup of obsolete task groups when scaling down. + */ + private boolean sendConfigUpdatesToTasks( + Map> newPartitionGroups, + Map latestTaskOffsetsOnPause + ) + throws InterruptedException, ExecutionException + { + log.info("Sending configuration updates to the following partition groups %s", newPartitionGroups); + List> updateFutures = new ArrayList<>(); + Map latestCommittedOffsets = getOffsetsFromMetadataStorage(); + for (Entry entry : activelyReadingTaskGroups.entrySet()) { + int taskGroupId = entry.getKey(); + TaskGroup existingTaskGroup = entry.getValue(); + + Set partitionsForThisTask = new HashSet<>(newPartitionGroups.getOrDefault(taskGroupId, Set.of())); + SeekableStreamIndexTaskIOConfig newIoConfig = createUpdatedTaskIoConfig( + partitionsForThisTask, + existingTaskGroup, + latestCommittedOffsets, + latestTaskOffsetsOnPause + ); + Map newStartingSequences = newIoConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap(); + TaskGroup newTaskGroup = existingTaskGroup.withStartingSequences(newStartingSequences); + for (String taskId : existingTaskGroup.taskIds()) { + log.info("Updating config for task [%s] with partitions [%s]", taskId, partitionsForThisTask); + TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest<>(newIoConfig, spec.getVersion().get(), latestTaskOffsetsOnPause); + updateFutures.add(taskClient.updateConfigAsync(taskId, updateRequest)); + } + activelyReadingTaskGroups.put(taskGroupId, newTaskGroup); + } + + if (updateFutures.isEmpty()) { + log.info("No configuration updates needed"); + return true; + } + + List> results = coalesceAndAwait(updateFutures); + boolean allSucceeded = results.stream().allMatch(result -> { + if (result.isValue()) { + Boolean value = result.valueOrThrow(); + return value != null && value; + } + return false; + }); + + if (allSucceeded) { + log.info("Successfully sent configuration updates to [%d] tasks", updateFutures.size()); + } else { + log.error("Some configuration updates failed"); + } + handleObsoleteTaskGroups(newPartitionGroups.keySet()); + + return allSucceeded; + } + + + /** + * Handles obsolete task groups when scaling down. + * Pauses tasks in task groups that are no longer needed and removes them from activelyReadingTaskGroups. + */ + private void handleObsoleteTaskGroups(Set newTaskGroupIds) + { + Set currentTaskGroupIds = new HashSet<>(activelyReadingTaskGroups.keySet()); + Set obsoleteTaskGroupIds = new HashSet<>(currentTaskGroupIds); + obsoleteTaskGroupIds.removeAll(newTaskGroupIds); + + if (obsoleteTaskGroupIds.isEmpty()) { + log.debug("No obsolete task groups to clean up"); + return; + } + + log.info("Handling obsolete task groups during scaling down: %s", obsoleteTaskGroupIds); + + for (Integer obsoleteTaskGroupId : obsoleteTaskGroupIds) { + TaskGroup obsoleteTaskGroup = activelyReadingTaskGroups.get(obsoleteTaskGroupId); + if (obsoleteTaskGroup != null) { + log.info("Gracefully shutting down tasks in obsolete task group [%d]: %s", obsoleteTaskGroupId, obsoleteTaskGroup.taskIds()); + + // Gracefully shut down all tasks in the obsolete task group + for (String taskId : obsoleteTaskGroup.taskIds()) { + try { + killTaskWithSuccess(taskId, "Gracefully shutting down task in obsolete task group [%d] during scale down", obsoleteTaskGroupId); + log.info("Requested graceful shutdown for task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); + } + catch (Exception e) { + log.error(e, "Failed to gracefully shut down task [%s] in obsolete task group [%d]", taskId, obsoleteTaskGroupId); + } + } + + // Remove the task group from activelyReadingTaskGroups + // The supervisor's normal run cycle will handle shutdown of these paused tasks + activelyReadingTaskGroups.remove(obsoleteTaskGroupId); + log.info("Removed obsolete task group [%d] from activelyReadingTaskGroups", obsoleteTaskGroupId); + } + } + } + + /** + * Updates the partition groups mapping for perpetual tasks without clearing other allocation info. + */ + private void updatePartitionGroupsForPerpetualTasks(Map> newPartitionGroups) + { + log.info("Updating partition groups mapping for perpetual tasks"); + + // Update the partition groups mapping + partitionGroups.clear(); + partitionGroups.putAll(newPartitionGroups); + + // Update partition offsets for new partitions if needed + for (PartitionIdType partition : partitionIds) { + partitionOffsets.putIfAbsent(partition, getNotSetMarker()); + } + + log.info("Updated partition groups: %s", partitionGroups); + } + + /** + * Creates an updated IOConfig for a task with new partition assignments. + */ + protected abstract SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ); + + private void changeTaskCountInIOConfig(int desiredActiveTaskCount) { ioConfig.setTaskCount(desiredActiveTaskCount); + spec.setVersion(DateTimes.nowUtc().toString()); try { Optional supervisorManager = taskMaster.getSupervisorManager(); if (supervisorManager.isPresent()) { @@ -741,6 +1046,7 @@ public String getType() protected class CheckpointNotice implements Notice { + private static final int DEFAULT_REENTRY_DELAY = 1500; private final int taskGroupId; private final SeekableStreamDataSourceMetadata checkpointMetadata; private static final String TYPE = "checkpoint_notice"; @@ -766,13 +1072,12 @@ public void handle() throws ExecutionException, InterruptedException // check validity of previousCheckpoint int index = checkpoints.size(); + // latest checkpoints are being compared first. for (int sequenceId : checkpoints.descendingKeySet()) { Map checkpoint = checkpoints.get(sequenceId); // We have already verified the stream of the current checkpoint is same with that in ioConfig. // See checkpoint(). - if (checkpoint.equals(checkpointMetadata.getSeekableStreamSequenceNumbers() - .getPartitionSequenceNumberMap() - )) { + if (isCheckpointSignatureValid(checkpoint, checkpointMetadata)) { break; } index--; @@ -790,7 +1095,10 @@ public void handle() throws ExecutionException, InterruptedException taskGroup.addNewCheckpoint(newCheckpoint); log.info("Handled checkpoint notice, new checkpoint is [%s] for taskGroup [%s]", newCheckpoint, taskGroupId); } else { + // It might be possible that the task has not been discovered to the taskgroup yet and have received a checkpoint before hand, + // For now, I will attempt to repush the checkpoint request in the handler. log.warn("New checkpoint is null for taskGroup [%s]", taskGroupId); + scheduledExec.schedule(() -> addNotice(this), DEFAULT_REENTRY_DELAY, TimeUnit.MILLISECONDS); } } } @@ -826,6 +1134,25 @@ public String getType() } } + /** + * This function verifies whether the checkpoint received from a task is valid or not. + * In non-perpetual tasks, the checkpoint is valid if it matches one of the partition <> sequence numbers in the task checkpoints. + * In perpetual tasks, this does not hold valid anymore because partition assignments can change during dynamic scaling. We'll compare + * it with supervisor level partition group's partition assignments now. + */ + private boolean isCheckpointSignatureValid( + Map checkpoint, + SeekableStreamDataSourceMetadata checkpointMetadata + ) + { + // Earlier checkpoints may no longer have the same partitions. + if (spec.usePersistentTasks()) { + return true; + } + + return checkpoint.equals(checkpointMetadata.getSeekableStreamSequenceNumbers().getPartitionSequenceNumberMap()); + } + // Map<{group id}, {actively reading task group}>; see documentation for TaskGroup class private final ConcurrentHashMap activelyReadingTaskGroups = new ConcurrentHashMap<>(); @@ -855,6 +1182,9 @@ public String getType() private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; protected final String dataSource; + private final AtomicBoolean isDynamicAllocationOngoing = new AtomicBoolean(false); + private int checkpointsToWaitFor = 0; + private Callable pendingConfigUpdateHook; private final Set subsequentlyDiscoveredPartitions = new HashSet<>(); private final TaskStorage taskStorage; @@ -940,7 +1270,7 @@ public SeekableStreamSupervisor( this.ioConfig = spec.getIoConfig(); this.autoScalerConfig = ioConfig.getAutoScalerConfig(); this.tuningConfig = spec.getTuningConfig(); - this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig(); + this.taskTuningConfig = this.tuningConfig.convertToTaskTuningConfig(spec.usePersistentTasks()); this.supervisorId = spec.getId(); this.exec = Execs.singleThreaded(StringUtils.encodeForFormat(supervisorTag)); this.scheduledExec = Execs.scheduledSingleThreaded(StringUtils.encodeForFormat(supervisorTag) + "-Scheduler-%d"); @@ -1479,7 +1809,7 @@ public Map> getStats() public List getParseErrors() { try { - if (spec.getSpec().getTuningConfig().convertToTaskTuningConfig().getMaxParseExceptions() <= 0) { + if (spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePersistentTasks()).getMaxParseExceptions() <= 0) { return ImmutableList.of(); } lastKnownParseErrors = getCurrentParseErrors(); @@ -1644,12 +1974,9 @@ private List getCurrentParseErrors() } } - SeekableStreamIndexTaskTuningConfig ss = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(); - SeekableStreamSupervisorIOConfig oo = spec.getSpec().getIOConfig(); - // store a limited number of parse exceptions, keeping the most recent ones - int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig().getMaxSavedParseExceptions() * - spec.getSpec().getIOConfig().getTaskCount(); + int parseErrorLimit = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePersistentTasks()).getMaxSavedParseExceptions() * + spec.getIoConfig().getTaskCount(); parseErrorLimit = Math.min(parseErrorLimit, parseErrorsTreeSet.size()); final List limitedParseErrors = new ArrayList<>(); @@ -1716,6 +2043,10 @@ public TaskGroup addTaskGroupToPendingCompletionTaskGroup( @VisibleForTesting public void runInternal() { + if (isDynamicAllocationOngoing.get()) { + log.info("Skipping run because dynamic allocation is ongoing."); + return; + } try { possiblyRegisterListener(); @@ -2089,6 +2420,19 @@ private void discoverTasks() throws ExecutionException, InterruptedException taskCount++; @SuppressWarnings("unchecked") final SeekableStreamIndexTask seekableStreamIndexTask = (SeekableStreamIndexTask) task; + SeekableStreamIndexTaskIOConfig taskIoConfig; + final String taskSpecVersion; + if (spec.usePersistentTasks()) { + // Fetch the current ioConfig being run on the task + final TaskConfigResponse runningConfig = FutureUtils.get( + taskClient.getTaskConfigAsync(task.getId()), true); + taskIoConfig = runningConfig.getIoConfig(); + taskSpecVersion = runningConfig.getSupervisorSpecVersion(); + } else { + taskSpecVersion = ""; + taskIoConfig = seekableStreamIndexTask.getIOConfig(); + } + final String taskId = task.getId(); // Check if the task has any inactive partitions. If so, terminate the task. Even if some of the @@ -2096,10 +2440,10 @@ private void discoverTasks() throws ExecutionException, InterruptedException // to more rapidly ensure that all active partitions are evenly distributed and being read, and to avoid // having to map expired partitions which are no longer tracked in partitionIds to a task group. if (supportsPartitionExpiration()) { - Set taskPartitions = seekableStreamIndexTask.getIOConfig() - .getStartSequenceNumbers() - .getPartitionSequenceNumberMap() - .keySet(); + Set taskPartitions = taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .keySet(); Set inactivePartitionsInTask = Sets.difference( taskPartitions, new HashSet<>(partitionIds) @@ -2122,11 +2466,11 @@ private void discoverTasks() throws ExecutionException, InterruptedException // state, we will permit it to complete even if it doesn't match our current partition allocation to support // seamless schema migration. - Iterator it = seekableStreamIndexTask.getIOConfig() - .getStartSequenceNumbers() - .getPartitionSequenceNumberMap() - .keySet() - .iterator(); + Iterator it = taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .keySet() + .iterator(); final Integer taskGroupId = (it.hasNext() ? getTaskGroupIdForPartition(it.next()) : null); if (taskGroupId != null) { @@ -2150,19 +2494,19 @@ public Boolean apply(Pair addDiscoveredTaskToPendingCompletionTaskGroups( - getTaskGroupIdForPartition( - partition), - taskId, - seekableStreamIndexTask.getIOConfig() - .getStartSequenceNumbers() - .getPartitionSequenceNumberMap() - )); + taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + .keySet() + .forEach( + partition -> addDiscoveredTaskToPendingCompletionTaskGroups( + getTaskGroupIdForPartition( + partition), + taskId, + taskIoConfig + .getStartSequenceNumbers() + .getPartitionSequenceNumberMap() + )); // update partitionGroups with the publishing task's sequences (if they are greater than what is // existing) so that the next tasks will start reading from where this task left off. @@ -2202,10 +2546,10 @@ public Boolean apply(Pair> entry : tasksToResume.entrySet()) { + for (Entry> entry : tasksToResume.entrySet()) { String taskId = entry.getKey(); ListenableFuture future = entry.getValue(); future.addListener( @@ -2710,7 +3054,7 @@ public Void apply(@Nullable Boolean result) * @return true if the task was created by the current supervisor */ @VisibleForTesting - public boolean isTaskCurrent(int taskGroupId, String taskId, Map activeTaskMap) + public boolean isTaskCurrent(int taskGroupId, String taskId, Map activeTaskMap, String version) { final Task genericTask; @@ -2728,6 +3072,11 @@ public boolean isTaskCurrent(int taskGroupId, String taskId, Map a SeekableStreamIndexTask task = (SeekableStreamIndexTask) genericTask; + // If the persistent tasks are present, we should use the version. + if (spec.usePersistentTasks()) { + return version.equals(spec.getVersion().get()); + } + // We recompute the sequence name hash for the supervisor's own configuration and compare this to the hash created // by rehashing the task's sequence name using the most up-to-date class definitions of tuning config and // data schema. Recomputing both ensures that forwards-compatible tasks won't be killed (which would occur @@ -2762,6 +3111,7 @@ public boolean isTaskCurrent(int taskGroupId, String taskId, Map a } } + @VisibleForTesting public String generateSequenceName( Map startPartitions, @@ -3301,6 +3651,10 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException } else { stopTasksEarly = false; } + // If using perpetually running tasks, we should not stop tasks based on duration + if (spec.usePersistentTasks()) { + return; + } final AtomicInteger numStoppedTasks = new AtomicInteger(); // Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing @@ -3529,13 +3883,24 @@ public ListenableFuture> apply(List task = iTasks.next(); String taskId = task.getKey(); TaskData taskData = task.getValue(); + final String version = spec.usePersistentTasks() ? getRunningSpecVersionOnTask(taskId) : spec.getVersion().get(); // stop and remove bad tasks from the task group - if (!isTaskCurrent(groupId, taskId, activeTaskMap)) { + if (!isTaskCurrent(groupId, taskId, activeTaskMap, version)) { log.info("Stopping task[%s] as it does not match the expected sequence range and ingestion spec.", taskId); futures.add(stopTask(taskId, false)); iTasks.remove(); @@ -3750,6 +4116,19 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep coalesceAndAwait(futures); } + private String getRunningSpecVersionOnTask(String taskId) + { + try { + final TaskConfigResponse runningConfig = FutureUtils.get( + taskClient.getTaskConfigAsync(taskId), true); + return runningConfig.getSupervisorSpecVersion(); + } + catch (InterruptedException | ExecutionException e) { + log.warn("Interrupted while fetching running spec version for task[%s]", taskId); + return ""; + } + } + private void checkIfStreamInactiveAndTurnSupervisorIdle() { if (!idleConfig.isEnabled() || spec.isSuspended()) { @@ -4661,7 +5040,7 @@ protected void emitLag() final LagStats lagStats = aggregatePartitionLags(partitionLags); Map metricTags = spec.getContextValue(DruidMetrics.TAGS); - for (Map.Entry entry : partitionLags.entrySet()) { + for (Entry entry : partitionLags.entrySet()) { emitter.emit( ServiceMetricEvent.builder() .setDimension(DruidMetrics.SUPERVISOR_ID, supervisorId) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java index 967652673f7c..5f96633ea831 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java @@ -37,6 +37,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.NoopTaskAutoScaler; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -45,6 +46,7 @@ import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.Optional; public abstract class SeekableStreamSupervisorSpec implements SupervisorSpec { @@ -78,6 +80,8 @@ private static SeekableStreamSupervisorIngestionSpec checkIngestionSchema( protected final DruidMonitorSchedulerConfig monitorSchedulerConfig; private final boolean suspended; protected final SupervisorStateManagerConfig supervisorStateManagerConfig; + protected final boolean usePersistentTasks; + protected String version; /** * Base constructor for SeekableStreamSupervisors. @@ -97,7 +101,8 @@ public SeekableStreamSupervisorSpec( ServiceEmitter emitter, DruidMonitorSchedulerConfig monitorSchedulerConfig, RowIngestionMetersFactory rowIngestionMetersFactory, - SupervisorStateManagerConfig supervisorStateManagerConfig + SupervisorStateManagerConfig supervisorStateManagerConfig, + @Nullable Boolean usePersistentTasks ) { this.ingestionSchema = checkIngestionSchema(ingestionSchema); @@ -117,6 +122,8 @@ public SeekableStreamSupervisorSpec( this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.suspended = suspended != null ? suspended : false; this.supervisorStateManagerConfig = supervisorStateManagerConfig; + this.usePersistentTasks = Configs.valueOrDefault(usePersistentTasks, false); + this.version = DateTimes.nowUtc().toString(); } @JsonProperty @@ -144,6 +151,12 @@ public SeekableStreamSupervisorIOConfig getIoConfig() return ingestionSchema.getIOConfig(); } + @JsonProperty + public boolean usePersistentTasks() + { + return usePersistentTasks; + } + @Nullable @JsonProperty public Map getContext() @@ -226,6 +239,16 @@ public boolean isSuspended() return suspended; } + public Optional getVersion() + { + return Optional.of(this.version); + } + + public void setVersion(String version) + { + this.version = version; + } + /** * Default implementation that prevents unsupported evolution of the supervisor spec *
    @@ -256,5 +279,4 @@ public void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcept } protected abstract SeekableStreamSupervisorSpec toggleSuspend(boolean suspend); - } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java index d1f10e77efa8..7f7e66506d77 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorTuningConfig.java @@ -55,5 +55,5 @@ static Duration defaultDuration(final Period period, final String theDefault) @JsonProperty Duration getOffsetFetchPeriod(); - SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks); } diff --git a/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java b/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java index 23e315aac0c6..1b871db80b69 100644 --- a/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java +++ b/indexing-service/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java @@ -189,4 +189,12 @@ default List getTaskInfos( */ void populateTaskTypeAndGroupIdAsync(); + + /** + * Update the task entry with the given id. + */ + void update( + String id, + @NotNull Task entry + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index b7190edf6518..9ce437912f38 100644 --- a/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/indexing-service/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -146,6 +146,26 @@ public void insert( } } + @Override + public void update( + final String id, + final @NotNull Task entry + ) + { + try { + getConnector().retryWithHandle( + handle -> updateEntryWithHandle(handle, id, entry), + this::isTransientDruidException + ); + } + catch (CallbackFailedException e) { + propagateAsRuntimeException(e.getCause()); + } + catch (Exception e) { + propagateAsRuntimeException(e); + } + } + private void propagateAsRuntimeException(Throwable t) { Throwables.propagateIfPossible(t); @@ -192,6 +212,28 @@ private Void insertEntryWithHandle( } } + private Void updateEntryWithHandle( + Handle handle, + String entryId, + Object entry + ) + { + try { + final String sql = StringUtils.format( + "UPDATE %s SET payload = :payload WHERE id = :id", + getEntryTable() + ); + handle.createStatement(sql) + .bind("id", entryId) + .bind("payload", jsonMapper.writeValueAsBytes(entry)) + .execute(); + return null; + } + catch (Throwable t) { + throw wrapInDruidException(entryId, t); + } + } + private boolean isTransientDruidException(Throwable t) { if (t instanceof CallbackFailedException) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index 3ee1fa065b28..662966067ac1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -744,6 +744,12 @@ private TestTask(String id, Interval interval, Map context) this.interval = interval; } + private TestTask(String id, String dataSource, Interval interval, Map context) + { + super(id, dataSource, context, IngestionMode.NONE); + this.interval = interval; + } + @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java index 72222ade56d6..1a1c2c7a5f19 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskClientAsyncImplTest.java @@ -565,6 +565,148 @@ public void test_getParseErrorsAsync_empty() throws Exception Assert.assertNull(client.getParseErrorsAsync(TASK_ID).get()); } + @Test + public void test_pauseAndCheckpointAsync_immediateOk() throws Exception + { + final Map offsets = ImmutableMap.of(1, 3L); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(offsets) + ); + + Assert.assertEquals(offsets, client.pauseAndCheckpointAsync(TASK_ID).get()); + } + + @Test + public void test_pauseAndCheckpointAsync_immediateBadStatus() throws Exception + { + final Map offsets = ImmutableMap.of(1, 3L); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + HttpResponseStatus.CONTINUE, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(offsets) + ); + + final ExecutionException e = Assert.assertThrows( + ExecutionException.class, + () -> client.pauseAndCheckpointAsync(TASK_ID).get() + ); + + MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IllegalStateException.class)); + MatcherAssert.assertThat( + e.getCause().getMessage(), + CoreMatchers.startsWith("Pause & Checkpoint request for task [the-task] failed with response [100 Continue]") + ); + } + + @Test + public void test_pauseAndCheckpointAsync_oneIteration() throws Exception + { + final Map offsets = ImmutableMap.of(1, 3L); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + HttpResponseStatus.ACCEPTED, + Collections.emptyMap(), + ByteArrays.EMPTY_ARRAY + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/status").timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(SeekableStreamIndexTaskRunner.Status.PAUSED) + ).expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/offsets/current").timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + jsonMapper.writeValueAsBytes(offsets) + ); + + Assert.assertEquals(offsets, client.pauseAndCheckpointAsync(TASK_ID).get()); + } + + @Test + public void test_pauseAndCheckpointAsync_notAvailable() throws Exception + { + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/pauseAndCheckpoint").timeout(httpTimeout), + new ServiceNotAvailableException(TASK_ID) + ); + + Assert.assertEquals(Collections.emptyMap(), client.pauseAndCheckpointAsync(TASK_ID).get()); + } + + @Test + public void test_updateConfigAsync_success() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/config") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + HttpResponseStatus.OK, + Collections.emptyMap(), + ByteArrays.EMPTY_ARRAY + ); + + Assert.assertEquals(true, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + + @Test + public void test_updateConfigAsync_httpError() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); + + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/config") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + new HttpResponseException( + new StringFullResponseHolder( + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.SERVICE_UNAVAILABLE), + StandardCharsets.UTF_8 + ) + ) + ); + + Assert.assertEquals(false, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + + @Test + public void test_updateConfigAsync_notAvailable() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); + + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/config") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + new ServiceNotAvailableException(TASK_ID) + ); + + Assert.assertEquals(false, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + + @Test + public void test_updateConfigAsync_closed() throws Exception + { + final TaskConfigUpdateRequest updateRequest = new TaskConfigUpdateRequest(null, null, null); + + serviceClient.expectAndThrow( + new RequestBuilder(HttpMethod.POST, "/config") + .content(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(updateRequest)) + .timeout(httpTimeout), + new ServiceClosedException(TASK_ID) + ); + + Assert.assertEquals(false, client.updateConfigAsync(TASK_ID, updateRequest).get()); + } + @Test public void test_serviceLocator_unknownTask() throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index ec71cec89c18..25d106aebca7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -375,7 +375,7 @@ public TestSeekableStreamIndexTask( SeekableStreamIndexTaskIOConfig ioConfig ) { - super(id, null, null, dataSchema, tuningConfig, ioConfig, null, null); + super(id, null, null, dataSchema, tuningConfig, ioConfig, null, null, null, "v1"); } @Override @@ -395,6 +395,12 @@ protected RecordSupplier newTaskRecordSupplier(final { return null; } + + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return null; + } } private static class TestSeekableStreamIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index e96e38873189..658796adf301 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -366,6 +366,18 @@ public LagStats computeLagStats() return new LagStats(0, 0, 0); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ) + { + // dummy implementation + return null; + } + @Override public int getPartitionCount() { @@ -424,7 +436,8 @@ public TestSeekableStreamSupervisorSpec( emitter, monitorSchedulerConfig, rowIngestionMetersFactory, - supervisorStateManagerConfig + supervisorStateManagerConfig, + null ); this.supervisor = supervisor; @@ -496,7 +509,7 @@ public Duration getOffsetFetchPeriod() } @Override - public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new SeekableStreamIndexTaskTuningConfig( null, @@ -742,11 +755,14 @@ public void testSeekableStreamSupervisorSpecWithScaleOut() throws InterruptedExc EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig(1, true)).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -803,6 +819,7 @@ public void testSeekableStreamSupervisorSpecWithScaleOutAlreadyAtMax() throws In EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -862,6 +879,7 @@ public void test_dynamicAllocationNotice_skipsScalingAndEmitsReason_ifTasksArePu EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -928,6 +946,7 @@ public void testSeekableStreamSupervisorSpecWithNoScalingOnIdleSupervisor() thro EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -973,11 +992,14 @@ public void testSeekableStreamSupervisorSpecWithScaleOutSmallPartitionNumber() t EasyMock.expect(spec.getId()).andReturn(SUPERVISOR).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig(1, true)).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1024,6 +1046,9 @@ public void testSeekableStreamSupervisorSpecWithScaleIn() throws InterruptedExce EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1074,6 +1099,9 @@ public void testSeekableStreamSupervisorSpecWithScaleInThresholdGreaterThanParti EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); + EasyMock.expect(spec.getVersion()).andReturn(java.util.Optional.of("v1")).anyTimes(); + spec.setVersion(EasyMock.anyString()); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1130,6 +1158,7 @@ public void testSeekableStreamSupervisorSpecWithScaleInAlreadyAtMin() throws Int EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(seekableStreamSupervisorIOConfig).anyTimes(); @@ -1210,6 +1239,7 @@ public void testSeekableStreamSupervisorSpecWithScaleDisable() throws Interrupte EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.replay(spec); EasyMock.expect(ingestionSchema.getIOConfig()).andReturn(this.seekableStreamSupervisorIOConfig).anyTimes(); @@ -1279,7 +1309,8 @@ public void testEnablingIdleBeviourPerSupervisorWithOverlordConfigEnabled() null, null, null, - supervisorStateManagerConfig + supervisorStateManagerConfig, + null ) { @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java new file mode 100644 index 000000000000..7ff448833614 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/TaskConfigUpdateRequestTest.java @@ -0,0 +1,54 @@ +/* + * 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; + +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +public class TaskConfigUpdateRequestTest +{ + @Test + public void testTaskConfigUpdateRequest() + { + SeekableStreamIndexTaskIOConfig mockIoConfig = EasyMock.createMock(SeekableStreamIndexTaskIOConfig.class); + EasyMock.replay(mockIoConfig); + + TaskConfigUpdateRequest request = new TaskConfigUpdateRequest(mockIoConfig, null, null); + Assert.assertEquals(mockIoConfig, request.getIoConfig()); + + TaskConfigUpdateRequest nullRequest = new TaskConfigUpdateRequest(null, null, null); + Assert.assertNull(nullRequest.getIoConfig()); + + TaskConfigUpdateRequest request2 = new TaskConfigUpdateRequest(mockIoConfig, null, null); + Assert.assertEquals(request, request2); + Assert.assertEquals(request.hashCode(), request2.hashCode()); + + Assert.assertNotEquals(request, nullRequest); + Assert.assertNotEquals(request.hashCode(), nullRequest.hashCode()); + + String toString = request.toString(); + Assert.assertTrue(toString.contains("TaskConfigUpdateRequest")); + Assert.assertTrue(toString.contains("ioConfig")); + Assert.assertTrue(toString.contains("supervisorSpecVersion")); + + EasyMock.verify(mockIoConfig); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java new file mode 100644 index 000000000000..363f4c0394bb --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/common/OrderedSequenceNumberTest.java @@ -0,0 +1,207 @@ +/* + * 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.common; + +import org.junit.Assert; +import org.junit.Test; + +import javax.validation.constraints.NotNull; + +public class OrderedSequenceNumberTest +{ + @Test + public void test_isMoreToReadBeforeReadingRecord_exclusiveEnd_lessThan() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertTrue("Should have more to read when current < end with exclusive end", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void test_isMoreToReadBeforeReadingRecord_exclusiveEnd_equalTo() + { + TestSequenceNumber current = new TestSequenceNumber(10L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should NOT have more to read when current == end with exclusive end", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_exclusiveEnd_greaterThan() + { + TestSequenceNumber current = new TestSequenceNumber(15L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should NOT have more to read when current > end with exclusive end", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_inclusiveEnd_lessThan() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertTrue("Should have more to read when current < end with inclusive end", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_inclusiveEnd_equalTo() + { + TestSequenceNumber current = new TestSequenceNumber(10L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertTrue("Should have more to read when current == end with inclusive end", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_inclusiveEnd_greaterThan() + { + TestSequenceNumber current = new TestSequenceNumber(15L, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertFalse("Should NOT have more to read when current > end with inclusive end", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullEndSequenceNumber_exclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(null, false); + + Assert.assertFalse("Should return false when end sequence number is null", + current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullEndSequenceNumber_inclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(5L, false); + TestSequenceNumber end = new TestSequenceNumber(null, false); + + Assert.assertFalse("Should return false when end sequence number is null", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullCurrentSequenceNumber_exclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(null, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertThrows(NullPointerException.class, () -> current.isMoreToReadBeforeReadingRecord(end, true)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_nullCurrentSequenceNumber_inclusiveEnd() + { + TestSequenceNumber current = new TestSequenceNumber(null, false); + TestSequenceNumber end = new TestSequenceNumber(10L, false); + + Assert.assertThrows(NullPointerException.class, () -> current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_bothNull() + { + TestSequenceNumber current = new TestSequenceNumber(null, false); + TestSequenceNumber end = new TestSequenceNumber(null, false); + + Assert.assertFalse("Should return false when both sequence numbers are null", + current.isMoreToReadBeforeReadingRecord(end, true)); + Assert.assertFalse("Should return false when both sequence numbers are null", + current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_exceptionHandling() + { + TestExceptionSequenceNumber current = new TestExceptionSequenceNumber(5L, false); + TestExceptionSequenceNumber end = new TestExceptionSequenceNumber(10L, false); + + Assert.assertThrows(RuntimeException.class, () -> current.isMoreToReadBeforeReadingRecord(end, true)); + Assert.assertThrows(RuntimeException.class, () -> current.isMoreToReadBeforeReadingRecord(end, false)); + } + + @Test + public void testIsMoreToReadBeforeReadingRecord_differentExclusivityFlags() + { + TestSequenceNumber currentExclusive = new TestSequenceNumber(5L, true); + TestSequenceNumber currentInclusive = new TestSequenceNumber(5L, false); + TestSequenceNumber endExclusive = new TestSequenceNumber(10L, true); + TestSequenceNumber endInclusive = new TestSequenceNumber(10L, false); + + // Test different combinations with exclusive end + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endExclusive, true)); + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endInclusive, true)); + Assert.assertTrue("Should work with different exclusivity flags", + currentInclusive.isMoreToReadBeforeReadingRecord(endExclusive, true)); + + // Test different combinations with inclusive end + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endExclusive, false)); + Assert.assertTrue("Should work with different exclusivity flags", + currentExclusive.isMoreToReadBeforeReadingRecord(endInclusive, false)); + Assert.assertTrue("Should work with different exclusivity flags", + currentInclusive.isMoreToReadBeforeReadingRecord(endExclusive, false)); + } + + /** + * Test implementation of OrderedSequenceNumber for Long values + */ + private static class TestSequenceNumber extends OrderedSequenceNumber + { + public TestSequenceNumber(Long sequenceNumber, boolean isExclusive) + { + super(sequenceNumber, isExclusive); + } + + @Override + public int compareTo(@NotNull OrderedSequenceNumber o) + { + return this.get().compareTo(o.get()); + } + } + + /** + * Test implementation that throws exceptions on comparison + */ + private static class TestExceptionSequenceNumber extends OrderedSequenceNumber + { + public TestExceptionSequenceNumber(Long sequenceNumber, boolean isExclusive) + { + super(sequenceNumber, isExclusive); + } + + @Override + public int compareTo(@NotNull OrderedSequenceNumber o) + { + throw new RuntimeException("Comparison not supported"); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index e204a67cae84..275c0fbd0d6e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -207,6 +207,7 @@ public void setupTest() public void testRunning() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -249,6 +250,7 @@ public void testRunningStreamGetSequenceNumberReturnsNull() EasyMock.expect(recordSupplier.getAssignment()).andReturn(ImmutableSet.of(SHARD0_PARTITION)).anyTimes(); EasyMock.expect(recordSupplier.getLatestSequenceNumber(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -298,6 +300,7 @@ public void testRunningStreamGetSequenceNumberReturnsNull() public void testAddDiscoveredTaskToPendingCompletionTaskGroups() throws Exception { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -402,6 +405,7 @@ public void testAddDiscoveredTaskToPendingCompletionTaskGroups() throws Exceptio public void testAddDiscoveredTaskToPendingCompletionMultipleTaskGroups() throws Exception { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -473,6 +477,7 @@ public void testAddDiscoveredTaskToPendingCompletionMultipleTaskGroups() throws public void testConnectingToStreamFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)) .andThrow(new StreamException(new IllegalStateException(EXCEPTION_MSG))) .anyTimes(); @@ -528,6 +533,7 @@ public void testConnectingToStreamFail() public void testConnectingToStreamFailRecoveryFailRecovery() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)) .andThrow(new StreamException(new IllegalStateException())) .times(3); @@ -602,6 +608,7 @@ public void testConnectingToStreamFailRecoveryFailRecovery() public void testDiscoveringInitialTasksFailRecoveryFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).times(6); @@ -711,6 +718,7 @@ public Duration getEmissionDuration() } }).anyTimes(); EasyMock.expect(spec.getType()).andReturn("test").anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); @@ -784,6 +792,7 @@ public void testIdleOnStartUpAndTurnsToRunningAfterLagUpdates() EasyMock.reset(spec); EasyMock.expect(spec.getId()).andReturn(SUPERVISOR_ID).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getContextValue("tags")).andReturn("").anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(new SeekableStreamSupervisorIOConfig( @@ -859,6 +868,7 @@ public Duration getEmissionDuration() public void testCreatingTasksFailRecoveryFail() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andThrow(new IllegalStateException(EXCEPTION_MSG)).times(3); @@ -937,6 +947,7 @@ public void testCreatingTasksFailRecoveryFail() public void testSuspended() { EasyMock.expect(spec.isSuspended()).andReturn(true).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -976,6 +987,7 @@ public void testSuspended() public void testStopping() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -1020,6 +1032,7 @@ public void testStopGracefully() throws Exception EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); taskRunner.unregisterListener("testSupervisorId"); indexTaskClient.close(); @@ -1039,6 +1052,7 @@ public void testStopGracefully() throws Exception public void testStoppingGracefully() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.getActiveTasksForDatasource(DATASOURCE)).andReturn(Map.of()).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); @@ -1111,6 +1125,7 @@ public void testCheckpointForActiveTaskGroup() throws InterruptedException, Json EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() { @Override public Duration getEmissionDuration() @@ -1135,7 +1150,7 @@ public Duration getEmissionDuration() ioConfig ); - SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(false); TreeMap> sequenceOffsets = new TreeMap<>(); sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L)); @@ -1323,6 +1338,7 @@ public void testEarlyStoppingOfTaskGroupBasedOnStopTaskCount() throws Interrupte EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() @@ -1339,7 +1355,7 @@ public Duration getEmissionDuration() EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes(); EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes(); - SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(false); TreeMap> sequenceOffsets = new TreeMap<>(); sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L)); @@ -1554,6 +1570,7 @@ public void testSupervisorStopTaskGroupEarly() throws JsonProcessingException, I EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes(); EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andStubReturn(false); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig() { @Override @@ -1570,7 +1587,7 @@ public Duration getEmissionDuration() EasyMock.expect(streamingTaskRunner.getStatus()).andReturn(null); EasyMock.expect(streamingTaskRunner.getStatus()).andReturn(SeekableStreamIndexTaskRunner.Status.NOT_STARTED); - SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(); + SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig(false); TreeMap> sequenceOffsets = new TreeMap<>(); sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L)); @@ -1728,7 +1745,6 @@ public void testEmitRecordLag() throws Exception public void testEmitTimeLag() throws Exception { expectEmitterSupervisor(false); - CountDownLatch latch = new CountDownLatch(1); TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( latch, @@ -1761,7 +1777,6 @@ public void testEmitTimeLag() throws Exception public void testEmitNoticesQueueSize() throws Exception { expectEmitterSupervisor(false); - CountDownLatch latch = new CountDownLatch(1); TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( latch, @@ -1825,7 +1840,6 @@ public void testEmitNoticesTime() throws Exception public void testEmitNoLagWhenSuspended() throws Exception { expectEmitterSupervisor(true); - CountDownLatch latch = new CountDownLatch(1); TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( latch, @@ -1860,6 +1874,7 @@ public void testEmitNoLagWhenSuspended() throws Exception public void testGetStats() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexTaskClient.getMovingAveragesAsync("task1")) .andReturn(Futures.immediateFuture(ImmutableMap.of("prop1", "val1"))) .times(1); @@ -1907,6 +1922,7 @@ public void testGetStats() public void testSupervisorResetAllWithCheckpoints() throws InterruptedException { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.deleteDataSourceMetadata(SUPERVISOR_ID)).andReturn( true ); @@ -1960,6 +1976,7 @@ public void testSupervisorResetOneTaskSpecificOffsetsWithCheckpoints() throws In ) ) ); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata(SUPERVISOR_ID, new TestSeekableStreamDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>( STREAM, @@ -2006,7 +2023,7 @@ public void testSupervisorResetOneTaskSpecificOffsetsWithCheckpoints() throws In public void testRegisterNewVersionOfPendingSegment() { EasyMock.expect(spec.isSuspended()).andReturn(false); - + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); Capture captured0 = Capture.newInstance(CaptureType.FIRST); Capture captured1 = Capture.newInstance(CaptureType.FIRST); EasyMock.expect( @@ -2088,7 +2105,7 @@ public void testSupervisorResetSpecificOffsetsTasksWithCheckpoints() throws Inte final ImmutableMap checkpointOffsets = ImmutableMap.of("0", "5", "1", "6", "2", "100"); final ImmutableMap resetOffsets = ImmutableMap.of("0", "10", "1", "8"); final ImmutableMap expectedOffsets = ImmutableMap.of("0", "10", "1", "8", "2", "100"); - + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.isSuspended()).andReturn(false); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( @@ -2168,6 +2185,7 @@ public void testSupervisorResetOffsetsWithNoCheckpoints() throws InterruptedExce final ImmutableMap expectedOffsets = ImmutableMap.copyOf(resetOffsets); EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn(null); EasyMock.expect(indexerMetadataStorageCoordinator.insertDataSourceMetadata(SUPERVISOR_ID, new TestSeekableStreamDataSourceMetadata( @@ -2241,6 +2259,7 @@ public void testSupervisorResetWithNoPartitions() throws IOException, Interrupte final ImmutableMap expectedOffsets = ImmutableMap.of("0", "5", "1", "6"); EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( new TestSeekableStreamDataSourceMetadata( @@ -2306,6 +2325,7 @@ public void testSupervisorResetWithNewPartition() throws IOException, Interrupte final ImmutableMap expectedOffsets = ImmutableMap.of("0", "5", "1", "6", "2", "20"); EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.reset(indexerMetadataStorageCoordinator); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(SUPERVISOR_ID)).andReturn( new TestSeekableStreamDataSourceMetadata( @@ -2369,6 +2389,7 @@ public void testSupervisorResetWithNewPartition() throws IOException, Interrupte public void testSupervisorNoResetDataSourceMetadata() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2408,6 +2429,7 @@ public void testSupervisorNoResetDataSourceMetadata() public void testSupervisorResetWithInvalidStartSequenceMetadata() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2458,6 +2480,7 @@ public void testSupervisorResetWithInvalidStartSequenceMetadata() public void testSupervisorResetInvalidStream() { EasyMock.expect(spec.isSuspended()).andReturn(false); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); replayAll(); final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor(); @@ -2504,7 +2527,6 @@ public void testSupervisorResetInvalidStream() public void testStaleOffsetsNegativeLagNotEmitted() throws Exception { expectEmitterSupervisor(false); - CountDownLatch latch = new CountDownLatch(1); final TestEmittingTestSeekableStreamSupervisor supervisor = new TestEmittingTestSeekableStreamSupervisor( @@ -2547,6 +2569,7 @@ private void validateSupervisorStateAfterResetOffsets( public void testScheduleReporting() { EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); DruidMonitorSchedulerConfig config = new DruidMonitorSchedulerConfig(); EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(config).times(2); ScheduledExecutorService executorService = EasyMock.createMock(ScheduledExecutorService.class); @@ -2649,6 +2672,7 @@ private void expectEmitterSupervisor(boolean suspended) spec = createMock(SeekableStreamSupervisorSpec.class); EasyMock.expect(spec.getId()).andReturn(SUPERVISOR_ID).anyTimes(); EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); + EasyMock.expect(spec.usePersistentTasks()).andReturn(false).anyTimes(); EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); EasyMock.expect(spec.getIoConfig()).andReturn(new SeekableStreamSupervisorIOConfig( "stream", @@ -2863,7 +2887,7 @@ public Duration getOffsetFetchPeriod() } @Override - public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() + public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig(Boolean usePerpetuallyRunningTasks) { return new SeekableStreamIndexTaskTuningConfig( null, @@ -2955,7 +2979,9 @@ public TestSeekableStreamIndexTask( tuningConfig, ioConfig, context, - groupId + groupId, + null, + "v1" ); this.streamingTaskRunner = streamingTaskRunner; } @@ -2973,6 +2999,12 @@ protected RecordSupplier newTaskRecordSupplier(final return recordSupplier; } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return this; + } + @Override public String getType() { @@ -2997,6 +3029,18 @@ private BaseTestSeekableStreamSupervisor() ); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ) + { + // dummy implementation + return null; + } + @Override protected String baseTaskName() { @@ -3212,6 +3256,17 @@ public LagStats computeLagStats() return new LagStats(0, 0, 0); } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ) + { + return null; + } + @Override protected Map getLatestSequencesFromStream() { @@ -3249,6 +3304,17 @@ private class TestEmittingTestSeekableStreamSupervisor extends BaseTestSeekableS this.partitionsTimeLag = partitionsTimeLag; } + @Override + protected SeekableStreamIndexTaskIOConfig createUpdatedTaskIoConfig( + Set partitions, + SeekableStreamSupervisor.TaskGroup existingTaskGroup, + Map latestCommittedOffsets, + Map latestTaskOffsetsOnPause + ) + { + return null; + } + @Nullable @Override protected Map getPartitionRecordLag() diff --git a/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java b/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java index 88db9e86f7ea..589e6b4bac34 100644 --- a/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/metadata/MetadataStorageActionHandlerTest.java @@ -148,6 +148,12 @@ public void populateTaskTypeAndGroupIdAsync() { } + + @Override + public void update(String id, Task entry) + { + + } }; } } diff --git a/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java b/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java index 604f682fb66a..3aee64437fce 100644 --- a/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java @@ -556,4 +556,30 @@ private void verifyTaskStatus(TaskStatus expected, TaskStatus actual) { Assert.assertEquals(expected, actual); } + + @Test + public void testUpdateTask() + { + Task originalTask = NoopTask.create(); + TaskStatus status = TaskStatus.running(originalTask.getId()); + final String entryId = originalTask.getId(); + + handler.insert(entryId, DateTimes.of("2014-01-01T00:00:00.123"), "testDataSource", originalTask, true, status, "type", "group"); + + Assert.assertEquals(Optional.of(originalTask), handler.getEntry(entryId)); + Assert.assertEquals("none", handler.getEntry(entryId).get().getDataSource()); + + Task updatedTask = new NoopTask(entryId, "updatedGroup", "updatedDataSource", 5000L, 0L, + ImmutableMap.of("testKey", "testValue")); + + handler.update(entryId, updatedTask); + + Optional retrievedTask = handler.getEntry(entryId); + Assert.assertTrue(retrievedTask.isPresent()); + Assert.assertEquals(entryId, retrievedTask.get().getId()); + Assert.assertEquals("updatedDataSource", retrievedTask.get().getDataSource()); + Assert.assertEquals("updatedGroup", retrievedTask.get().getGroupId()); + Assert.assertEquals("testValue", retrievedTask.get().getContextValue("testKey")); + Assert.assertEquals(5000L, ((NoopTask) retrievedTask.get()).getRunTime()); + } } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java index 9ff217d5404a..f1237be74745 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorSpec.java @@ -28,6 +28,7 @@ import javax.annotation.Nonnull; import java.util.List; +import java.util.Optional; import java.util.Set; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @@ -113,4 +114,9 @@ default void validateSpecUpdateTo(SupervisorSpec proposedSpec) throws DruidExcep { // The default implementation does not do any validation checks. } + + default Optional getVersion() + { + return Optional.empty(); + } } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java index 2882286de65e..192a5be72bc8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSupervisorManager.java @@ -89,6 +89,7 @@ public void insert(final String id, final SupervisorSpec spec) { dbi.withHandle( handle -> { + String created_date = spec.getVersion().orElse(DateTimes.nowUtc().toString()); handle.createStatement( StringUtils.format( "INSERT INTO %s (spec_id, created_date, payload) VALUES (:spec_id, :created_date, :payload)", @@ -96,7 +97,7 @@ public void insert(final String id, final SupervisorSpec spec) ) ) .bind("spec_id", id) - .bind("created_date", DateTimes.nowUtc().toString()) + .bind("created_date", created_date) .bind("payload", jsonMapper.writeValueAsBytes(spec)) .execute(); diff --git a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java index 4a6a3660c335..88e4f64afa58 100644 --- a/services/src/test/java/org/apache/druid/cli/CliPeonTest.java +++ b/services/src/test/java/org/apache/druid/cli/CliPeonTest.java @@ -266,7 +266,7 @@ private TestStreamingTask( ) { - super(id, supervisorId, taskResource, dataSchema, tuningConfig, ioConfig, context, groupId); + super(id, supervisorId, taskResource, dataSchema, tuningConfig, ioConfig, context, groupId, null, "v1"); } @Override @@ -281,6 +281,12 @@ protected RecordSupplier newTaskRecordSupplier(final return null; } + @Override + public SeekableStreamIndexTask withNewIoConfig(SeekableStreamIndexTaskIOConfig newIoConfig) + { + return this; + } + @Override public String getCurrentRunnerStatus() {