From 43cb678fb84b128936ea9bbfc7f3c7c06d71bd4b Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 27 Jun 2019 14:03:23 -0700 Subject: [PATCH 01/24] Add CliIndexer process type and initial task runner implementation --- .../security/basic/CommonCacheNotifier.java | 3 +- ...ementalPublishingKafkaIndexTaskRunner.java | 7 +- .../druid/indexing/kafka/KafkaIndexTask.java | 10 +- .../kafka/supervisor/KafkaSupervisor.java | 3 +- .../indexing/kafka/KafkaIndexTaskTest.java | 12 +- .../kafka/supervisor/KafkaSupervisorTest.java | 4 +- .../indexing/kinesis/KinesisIndexTask.java | 10 +- .../kinesis/KinesisIndexTaskRunner.java | 7 +- .../kinesis/supervisor/KinesisSupervisor.java | 3 +- .../kinesis/KinesisIndexTaskTest.java | 19 +- .../supervisor/KinesisSupervisorTest.java | 4 +- .../SingleFileTaskReportFileWriter.java | 53 ++ .../indexing/common/TaskReportFileWriter.java | 22 +- .../AppenderatorDriverRealtimeIndexTask.java | 31 +- .../indexing/common/task/CompactionTask.java | 19 +- .../indexing/common/task/HadoopIndexTask.java | 10 +- .../druid/indexing/common/task/IndexTask.java | 25 +- .../parallel/ParallelIndexSupervisorTask.java | 3 +- .../overlord/ThreadingTaskRunner.java | 710 ++++++++++++++++++ .../overlord/hrtr/HttpRemoteTaskRunner.java | 26 + .../SeekableStreamIndexTask.java | 10 +- .../SeekableStreamIndexTaskRunner.java | 22 +- .../indexing/worker/WorkerTaskManager.java | 3 +- .../druid/indexing/common/TestUtils.java | 3 + ...penderatorDriverRealtimeIndexTaskTest.java | 11 +- .../common/task/CompactionTaskRunTest.java | 15 +- .../common/task/CompactionTaskTest.java | 15 +- .../indexing/common/task/IndexTaskTest.java | 67 +- .../common/task/NoopTestTaskFileWriter.java | 6 +- .../indexing/common/task/TaskSerdeTest.java | 6 +- .../common/task/TestAppenderatorsManager.java | 152 ++++ .../SingleTaskBackgroundRunnerTest.java | 4 +- .../indexing/overlord/TaskLifecycleTest.java | 15 +- .../SeekableStreamSupervisorStateTest.java | 3 +- .../discovery/DruidNodeDiscoveryProvider.java | 6 +- .../org/apache/druid/discovery/NodeType.java | 3 +- .../UnifiedIndexerLifecycleHandler.java | 99 +++ .../appenderator/AppenderatorImpl.java | 43 +- .../appenderator/AppenderatorsManager.java | 127 ++++ ...DummyForInjectionAppenderatorsManager.java | 120 +++ .../PeonAppenderatorsManager.java | 158 ++++ .../appenderator/SinkQuerySegmentWalker.java | 6 + .../UnifiedIndexerAppenderatorsManager.java | 206 +++++ .../firehose/ChatHandlerResource.java | 4 +- .../druid/server/http/ClusterResource.java | 5 + .../jetty/ChatHandlerServerModule.java | 53 +- .../java/org/apache/druid/cli/CliIndexer.java | 306 ++++++++ .../apache/druid/cli/CliMiddleManager.java | 6 + .../org/apache/druid/cli/CliOverlord.java | 7 + .../java/org/apache/druid/cli/CliPeon.java | 11 +- .../main/java/org/apache/druid/cli/Main.java | 1 + 51 files changed, 2345 insertions(+), 129 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/UnifiedIndexerLifecycleHandler.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java create mode 100644 services/src/main/java/org/apache/druid/cli/CliIndexer.java diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java index f1e61ebfba75..98de3833717b 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java @@ -68,7 +68,8 @@ public class CommonCacheNotifier NodeType.HISTORICAL, NodeType.PEON, NodeType.ROUTER, - NodeType.MIDDLE_MANAGER + NodeType.MIDDLE_MANAGER, + NodeType.INDEXER ); private final DruidNodeDiscoveryProvider discoveryProvider; diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index bf0580cd7ad0..66399aac0e7a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.utils.CircularBuffer; @@ -70,7 +71,8 @@ public IncrementalPublishingKafkaIndexTaskRunner( AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, CircularBuffer savedParseExceptions, - RowIngestionMetersFactory rowIngestionMetersFactory + RowIngestionMetersFactory rowIngestionMetersFactory, + AppenderatorsManager appenderatorsManager ) { super( @@ -79,7 +81,8 @@ public IncrementalPublishingKafkaIndexTaskRunner( authorizerMapper, chatHandlerProvider, savedParseExceptions, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); this.task = task; } 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 b0b0c6be7470..d165ceb19576 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 @@ -29,6 +29,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -63,7 +64,8 @@ public KafkaIndexTask( @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject ObjectMapper configMapper + @JacksonInject ObjectMapper configMapper, + @JacksonInject AppenderatorsManager appenderatorsManager ) { super( @@ -76,7 +78,8 @@ public KafkaIndexTask( chatHandlerProvider, authorizerMapper, rowIngestionMetersFactory, - getFormattedGroupId(dataSchema.getDataSource(), TYPE) + getFormattedGroupId(dataSchema.getDataSource(), TYPE), + appenderatorsManager ); this.configMapper = configMapper; this.ioConfig = ioConfig; @@ -135,7 +138,8 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() authorizerMapper, chatHandlerProvider, savedParseExceptions, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); } 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 cdf133677bff..5963c84c028c 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 @@ -254,7 +254,8 @@ protected List> createIndexTasks( null, null, rowIngestionMetersFactory, - sortingMapper + sortingMapper, + null )); } return taskList; 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 c9506d6ef203..7b605731c334 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 @@ -51,9 +51,9 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.SegmentLoaderFactory; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; @@ -67,6 +67,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.kafka.test.TestBroker; @@ -140,6 +141,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; @@ -213,6 +215,7 @@ public class KafkaIndexTaskTest private Long maxTotalRows = null; private Period intermediateHandoffPeriod = null; + private AppenderatorsManager appenderatorsManager; private TaskToolboxFactory toolboxFactory; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private TaskStorage taskStorage; @@ -350,6 +353,7 @@ public void setupTest() throws IOException topic = getTopicName(); records = generateRecords(topic); reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); + appenderatorsManager = new TestAppenderatorsManager(); makeToolboxFactory(); } @@ -2485,7 +2489,8 @@ private KafkaIndexTask createTask( null, null, rowIngestionMetersFactory, - OBJECT_MAPPER + OBJECT_MAPPER, + appenderatorsManager ); task.setPollRetryMs(POLL_RETRY_MS); return task; @@ -2651,6 +2656,7 @@ public void close() final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig); + toolboxFactory = new TaskToolboxFactory( taskConfig, taskActionClientFactory, @@ -2676,7 +2682,7 @@ public void close() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new TaskReportFileWriter(reportsFile) + new SingleFileTaskReportFileWriter(reportsFile) ); } 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 aff5639cd0e0..4590a165b4e1 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 @@ -81,6 +81,7 @@ import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireDepartment; +import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -3506,7 +3507,8 @@ private KafkaIndexTask createKafkaIndexTask( null, null, rowIngestionMetersFactory, - objectMapper + objectMapper, + new DummyForInjectionAppenderatorsManager() ); } 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 f3dfe3bfcef0..970091826777 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 @@ -28,6 +28,7 @@ import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; @@ -50,7 +51,8 @@ public KinesisIndexTask( @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject AWSCredentialsConfig awsCredentialsConfig + @JacksonInject AWSCredentialsConfig awsCredentialsConfig, + @JacksonInject AppenderatorsManager appenderatorsManager ) { super( @@ -63,7 +65,8 @@ public KinesisIndexTask( chatHandlerProvider, authorizerMapper, rowIngestionMetersFactory, - getFormattedGroupId(dataSchema.getDataSource(), TYPE) + getFormattedGroupId(dataSchema.getDataSource(), TYPE), + appenderatorsManager ); this.awsCredentialsConfig = awsCredentialsConfig; } @@ -79,7 +82,8 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() authorizerMapper, chatHandlerProvider, savedParseExceptions, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index 335119a08b19..f6d7017ceee7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.utils.CircularBuffer; @@ -64,7 +65,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner chatHandlerProvider, CircularBuffer savedParseExceptions, - RowIngestionMetersFactory rowIngestionMetersFactory + RowIngestionMetersFactory rowIngestionMetersFactory, + AppenderatorsManager appenderatorsManager ) { super( @@ -73,7 +75,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner> createIndexTasks( null, null, rowIngestionMetersFactory, - awsCredentialsConfig + awsCredentialsConfig, + null )); } return taskList; 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 d8bf83ce2a39..89d2a97fd26c 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 @@ -57,9 +57,9 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.SegmentLoaderFactory; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; @@ -74,6 +74,7 @@ import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -139,6 +140,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; @@ -213,6 +215,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport private final Period intermediateHandoffPeriod = null; private int maxRecordsPerPoll; + private AppenderatorsManager appenderatorsManager; private TaskToolboxFactory toolboxFactory; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private TaskStorage taskStorage; @@ -293,6 +296,8 @@ public void setupTest() throws IOException, InterruptedException recordSupplier = mock(KinesisRecordSupplier.class); + appenderatorsManager = new TestAppenderatorsManager(); + // sleep required because of kinesalite Thread.sleep(500); makeToolboxFactory(); @@ -2678,7 +2683,8 @@ private KinesisIndexTask createTask( null, null, rowIngestionMetersFactory, - null + null, + appenderatorsManager ); } @@ -2837,6 +2843,7 @@ public void close() final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig(); dataSegmentPusherConfig.storageDirectory = getSegmentDirectory(); final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig); + toolboxFactory = new TaskToolboxFactory( taskConfig, taskActionClientFactory, @@ -2862,7 +2869,7 @@ public void close() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new TaskReportFileWriter(reportsFile) + new SingleFileTaskReportFileWriter(reportsFile) ); } @@ -3025,7 +3032,8 @@ private TestableKinesisIndexTask( @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, - @JacksonInject AWSCredentialsConfig awsCredentialsConfig + @JacksonInject AWSCredentialsConfig awsCredentialsConfig, + @JacksonInject AppenderatorsManager appenderatorsManager ) { super( @@ -3038,7 +3046,8 @@ private TestableKinesisIndexTask( chatHandlerProvider, authorizerMapper, rowIngestionMetersFactory, - awsCredentialsConfig + awsCredentialsConfig, + appenderatorsManager ); } 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 9b6f89341290..a19e38b09238 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 @@ -80,6 +80,7 @@ import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireDepartment; +import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -4258,7 +4259,8 @@ private KinesisIndexTask createKinesisIndexTask( null, null, rowIngestionMetersFactory, - null + null, + new DummyForInjectionAppenderatorsManager() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java new file mode 100644 index 000000000000..3bd0c1a85f70 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java @@ -0,0 +1,53 @@ +/* + * 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.common; + +import org.apache.commons.io.FileUtils; +import org.apache.druid.java.util.common.logger.Logger; + +import java.io.File; +import java.util.Map; + +public class SingleFileTaskReportFileWriter extends TaskReportFileWriter +{ + private static final Logger log = new Logger(SingleFileTaskReportFileWriter.class); + + private final File reportsFile; + + public SingleFileTaskReportFileWriter(File reportsFile) + { + this.reportsFile = reportsFile; + } + + @Override + public void write(String taskId, Map reports) + { + try { + final File reportsFileParent = reportsFile.getParentFile(); + if (reportsFileParent != null) { + FileUtils.forceMkdir(reportsFileParent); + } + objectMapper.writeValue(reportsFile, reports); + } + catch (Exception e) { + log.error(e, "Encountered exception in write()."); + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java index d9c9e81576e4..360456642e80 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java @@ -24,22 +24,34 @@ import org.apache.druid.java.util.common.logger.Logger; import java.io.File; +import java.util.HashMap; import java.util.Map; public class TaskReportFileWriter { private static final Logger log = new Logger(TaskReportFileWriter.class); - private final File reportsFile; - private ObjectMapper objectMapper; + private final Map taskReportFiles = new HashMap<>(); + protected ObjectMapper objectMapper; - public TaskReportFileWriter(File reportFile) + public void add(String taskId, File reportsFile) { - this.reportsFile = reportFile; + taskReportFiles.put(taskId, reportsFile); } - public void write(Map reports) + public void delete(String taskId) { + taskReportFiles.remove(taskId); + } + + public void write(String taskId, Map reports) + { + final File reportsFile = taskReportFiles.get(taskId); + if (reportsFile == null) { + log.error("Could not find report file for task[%s]", taskId); + return; + } + try { final File reportsFileParent = reportsFile.getParentFile(); if (reportsFileParent != null) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 81f4797d08f0..9ce27c89a41a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -72,7 +72,7 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; @@ -166,6 +166,9 @@ private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec) @JsonIgnore private String errorMsg; + @JsonIgnore + private AppenderatorsManager appenderatorsManager; + @JsonCreator public AppenderatorDriverRealtimeIndexTask( @JsonProperty("id") String id, @@ -174,7 +177,8 @@ public AppenderatorDriverRealtimeIndexTask( @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject AppenderatorsManager appenderatorsManager ) { super( @@ -195,6 +199,7 @@ public AppenderatorDriverRealtimeIndexTask( this.ingestionState = IngestionState.NOT_STARTED; this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); + this.appenderatorsManager = appenderatorsManager; } @Override @@ -270,9 +275,10 @@ public TaskStatus run(final TaskToolbox toolbox) log.warn("No chat handler detected"); } - - toolbox.getDataSegmentServerAnnouncer().announce(); - toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + if (appenderatorsManager.shouldTaskMakeNodeAnnouncements()) { + toolbox.getDataSegmentServerAnnouncer().announce(); + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + } driver.startJob(); @@ -369,7 +375,7 @@ public TaskStatus run(final TaskToolbox toolbox) log.makeAlert(e, "Exception aborted realtime processing[%s]", dataSchema.getDataSource()) .emit(); errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( getId(), errorMsg @@ -386,12 +392,14 @@ public TaskStatus run(final TaskToolbox toolbox) toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); - toolbox.getDataSegmentServerAnnouncer().unannounce(); - toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); + if (appenderatorsManager.shouldTaskMakeNodeAnnouncements()) { + toolbox.getDataSegmentServerAnnouncer().unannounce(); + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); + } } log.info("Job done!"); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } @@ -682,14 +690,15 @@ private DiscoveryDruidNode createDiscoveryDruidNode(TaskToolbox toolbox) ); } - private static Appenderator newAppenderator( + private Appenderator newAppenderator( final DataSchema dataSchema, final RealtimeAppenderatorTuningConfig tuningConfig, final FireDepartmentMetrics metrics, final TaskToolbox toolbox ) { - return Appenderators.createRealtime( + return appenderatorsManager.createRealtimeAppenderatorForTask( + getId(), dataSchema, tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 6a0ae54d8494..b62c2a60a601 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -74,6 +74,7 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.security.AuthorizerMapper; @@ -147,6 +148,9 @@ public class CompactionTask extends AbstractTask @JsonIgnore private List indexTaskSpecs; + @JsonIgnore + private AppenderatorsManager appenderatorsManager; + @JsonCreator public CompactionTask( @JsonProperty("id") final String id, @@ -167,7 +171,8 @@ public CompactionTask( @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, @JacksonInject CoordinatorClient coordinatorClient, @JacksonInject SegmentLoaderFactory segmentLoaderFactory, - @JacksonInject RetryPolicyFactory retryPolicyFactory + @JacksonInject RetryPolicyFactory retryPolicyFactory, + @JacksonInject AppenderatorsManager appenderatorsManager ) { super(getOrMakeId(id, TYPE, dataSource), null, taskResource, dataSource, context); @@ -194,6 +199,7 @@ public CompactionTask( this.coordinatorClient = coordinatorClient; this.segmentLoaderFactory = segmentLoaderFactory; this.retryPolicyFactory = retryPolicyFactory; + this.appenderatorsManager = appenderatorsManager; } @JsonProperty @@ -294,7 +300,8 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception getContext(), authorizerMapper, chatHandlerProvider, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager )) .collect(Collectors.toList()); } @@ -882,6 +889,7 @@ public static class Builder private final CoordinatorClient coordinatorClient; private final SegmentLoaderFactory segmentLoaderFactory; private final RetryPolicyFactory retryPolicyFactory; + private final AppenderatorsManager appenderatorsManager; @Nullable private Interval interval; @@ -908,7 +916,8 @@ public Builder( RowIngestionMetersFactory rowIngestionMetersFactory, CoordinatorClient coordinatorClient, SegmentLoaderFactory segmentLoaderFactory, - RetryPolicyFactory retryPolicyFactory + RetryPolicyFactory retryPolicyFactory, + AppenderatorsManager appenderatorsManager ) { this.dataSource = dataSource; @@ -919,6 +928,7 @@ public Builder( this.coordinatorClient = coordinatorClient; this.segmentLoaderFactory = segmentLoaderFactory; this.retryPolicyFactory = retryPolicyFactory; + this.appenderatorsManager = appenderatorsManager; } public Builder interval(Interval interval) @@ -990,7 +1000,8 @@ public CompactionTask build() rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 2571c576d2aa..7e8b5e49a880 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -254,7 +254,7 @@ public TaskStatus run(TaskToolbox toolbox) } errorMsg = Throwables.getStackTraceAsString(effectiveException); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( getId(), errorMsg @@ -318,7 +318,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception indexerSchema = determineConfigStatus.getSchema(); if (indexerSchema == null) { errorMsg = determineConfigStatus.getErrorMsg(); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( getId(), errorMsg @@ -365,7 +365,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception specVersion, version ); - toolbox.getTaskReportFileWriter().write(null); + toolbox.getTaskReportFileWriter().write(getId(), null); return TaskStatus.failure(getId()); } } @@ -404,14 +404,14 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception if (buildSegmentsStatus.getDataSegments() != null) { ingestionState = IngestionState.COMPLETED; toolbox.publishSegments(buildSegmentsStatus.getDataSegments()); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success( getId(), null ); } else { errorMsg = buildSegmentsStatus.getErrorMsg(); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( getId(), errorMsg diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 27df20a6a476..424684a3bcc0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -75,6 +75,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; @@ -176,6 +177,9 @@ private static String makeGroupId(boolean isAppendToExisting, String dataSource) @JsonIgnore private final RowIngestionMeters buildSegmentsMeters; + @JsonIgnore + private final AppenderatorsManager appenderatorsManager; + @JsonCreator public IndexTask( @JsonProperty("id") final String id, @@ -184,7 +188,8 @@ public IndexTask( @JsonProperty("context") final Map context, @JacksonInject AuthorizerMapper authorizerMapper, @JacksonInject ChatHandlerProvider chatHandlerProvider, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject AppenderatorsManager appenderatorsManager ) { this( @@ -196,7 +201,8 @@ public IndexTask( context, authorizerMapper, chatHandlerProvider, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); } @@ -209,7 +215,8 @@ public IndexTask( Map context, AuthorizerMapper authorizerMapper, ChatHandlerProvider chatHandlerProvider, - RowIngestionMetersFactory rowIngestionMetersFactory + RowIngestionMetersFactory rowIngestionMetersFactory, + AppenderatorsManager appenderatorsManager ) { super( @@ -234,6 +241,7 @@ public IndexTask( this.ingestionState = IngestionState.NOT_STARTED; this.determinePartitionsMeters = rowIngestionMetersFactory.createRowIngestionMeters(); this.buildSegmentsMeters = rowIngestionMetersFactory.createRowIngestionMeters(); + this.appenderatorsManager = appenderatorsManager; } @Override @@ -478,7 +486,7 @@ public TaskStatus run(final TaskToolbox toolbox) catch (Exception e) { log.error(e, "Encountered exception in %s.", ingestionState); errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( getId(), errorMsg @@ -1006,7 +1014,7 @@ private TaskStatus generateAndPublishSegments( if (published == null) { log.error("Failed to publish segments, aborting!"); errorMsg = "Failed to publish segments."; - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( getId(), errorMsg @@ -1020,7 +1028,7 @@ private TaskStatus generateAndPublishSegments( ); log.info("Published segments: %s", Lists.transform(published.getSegments(), DataSegment::getId)); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports()); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } } @@ -1098,14 +1106,15 @@ private static SegmentsAndMetadata awaitPublish( } } - private static Appenderator newAppenderator( + private Appenderator newAppenderator( FireDepartmentMetrics metrics, TaskToolbox toolbox, DataSchema dataSchema, IndexTuningConfig tuningConfig ) { - return Appenderators.createOffline( + return appenderatorsManager.createOfflineAppenderatorForTask( + getId(), dataSchema, tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 2a29726965c4..70f46fffc227 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -317,7 +317,8 @@ private TaskStatus runSequential(TaskToolbox toolbox) getContext(), authorizerMapper, chatHandlerProvider, - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ).run(toolbox); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java new file mode 100644 index 000000000000..253b3bcb20be --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -0,0 +1,710 @@ +/* + * 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.overlord; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.io.ByteSource; +import com.google.common.io.Files; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import com.google.inject.Inject; +import org.apache.commons.io.FileUtils; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexer.RunnerTaskState; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.server.DruidNode; +import org.apache.druid.tasklogs.TaskLogPusher; +import org.apache.druid.tasklogs.TaskLogStreamer; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; + +/** + * TaskRunner implemention for the Indexer task execution service, which runs all tasks in a single process. + * + * Two thread pools are used: + * - A task execution pool, sized to number of worker slots. This is used to execute the Task run() methods. + * - A control thread pool, sized to worker slots * 2. The control threads setup and submit work to the + * task execution pool, and are also responsible for running graceful shutdown on the Task objects. + * Only one shutdown per-task can be running at a given time, and there is one control thread per task, + * thus the pool has 2 * worker slots. + * + * Note that separate task logs are not supported, all task log entries will be written to the Indexer process log + * instead. + */ +public class ThreadingTaskRunner implements TaskRunner, TaskLogStreamer, QuerySegmentWalker +{ + private static final EmittingLogger log = new EmittingLogger(ThreadingTaskRunner.class); + + private static final String TASK_RESTORE_FILENAME = "restore.json"; + private final TaskToolboxFactory toolboxFactory; + private final TaskConfig taskConfig; + private final TaskLogPusher taskLogPusher; + private final DruidNode node; + private final ObjectMapper jsonMapper; + private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); + private final AppenderatorsManager appenderatorsManager; + private final TaskReportFileWriter taskReportFileWriter; + private final ListeningExecutorService taskExecutor; + private final ListeningExecutorService controlThreadExecutor; + + private volatile boolean stopping = false; + + /** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */ + private final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + + @Inject + public ThreadingTaskRunner( + TaskToolboxFactory toolboxFactory, + TaskConfig taskConfig, + WorkerConfig workerConfig, + TaskLogPusher taskLogPusher, + ObjectMapper jsonMapper, + AppenderatorsManager appenderatorsManager, + TaskReportFileWriter taskReportFileWriter, + @Self DruidNode node + ) + { + this.toolboxFactory = toolboxFactory; + this.taskConfig = taskConfig; + this.taskLogPusher = taskLogPusher; + this.jsonMapper = jsonMapper; + this.node = node; + this.appenderatorsManager = appenderatorsManager; + this.taskReportFileWriter = taskReportFileWriter; + this.taskExecutor = MoreExecutors.listeningDecorator( + Execs.multiThreaded(workerConfig.getCapacity(), "threading-task-runner-executor-%d") + ); + this.controlThreadExecutor = MoreExecutors.listeningDecorator( + Execs.multiThreaded(workerConfig.getCapacity() * 2, "threading-task-runner-control-%d") + ); + } + + @Override + public Optional streamTaskLog(String taskid, long offset) throws IOException + { + // task logs will appear in the main indexer log, streaming individual task logs is not supported + return Optional.absent(); + } + + @Override + public List>> restore() + { + final File restoreFile = getRestoreFile(); + final TaskRestoreInfo taskRestoreInfo; + if (restoreFile.exists()) { + try { + taskRestoreInfo = jsonMapper.readValue(restoreFile, TaskRestoreInfo.class); + } + catch (Exception e) { + log.error(e, "Failed to read restorable tasks from file[%s]. Skipping restore.", restoreFile); + return ImmutableList.of(); + } + } else { + return ImmutableList.of(); + } + + final List>> retVal = new ArrayList<>(); + for (final String taskId : taskRestoreInfo.getRunningTasks()) { + try { + final File taskFile = new File(taskConfig.getTaskDir(taskId), "task.json"); + final Task task = jsonMapper.readValue(taskFile, Task.class); + + if (!task.getId().equals(taskId)) { + throw new ISE("WTF?! Task[%s] restore file had wrong id[%s].", taskId, task.getId()); + } + + if (taskConfig.isRestoreTasksOnRestart() && task.canRestore()) { + log.info("Restoring task[%s].", task.getId()); + retVal.add(Pair.of(task, run(task))); + } + } + catch (Exception e) { + log.warn(e, "Failed to restore task[%s]. Trying to restore other tasks.", taskId); + } + } + + log.info("Restored %,d tasks.", retVal.size()); + + return retVal; + } + + @Override + public void start() + { + + } + + @Override + public void registerListener(TaskRunnerListener listener, Executor executor) + { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listener.getListenerId())) { + throw new ISE("Listener [%s] already registered", listener.getListenerId()); + } + } + + final Pair listenerPair = Pair.of(listener, executor); + + synchronized (tasks) { + for (ThreadingTaskRunnerWorkItem item : tasks.values()) { + TaskRunnerUtils.notifyLocationChanged(ImmutableList.of(listenerPair), item.getTaskId(), item.getLocation()); + } + + listeners.add(listenerPair); + log.info("Registered listener [%s]", listener.getListenerId()); + } + } + + @Override + public void unregisterListener(String listenerId) + { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listenerId)) { + listeners.remove(pair); + log.info("Unregistered listener [%s]", listenerId); + return; + } + } + } + + @Override + public ListenableFuture run(Task task) + { + synchronized (tasks) { + tasks.computeIfAbsent( + task.getId(), k -> + new ThreadingTaskRunnerWorkItem( + task, + controlThreadExecutor.submit( + new Callable() { + @Override + public TaskStatus call() + { + final String attemptUUID = UUID.randomUUID().toString(); + final File taskDir = taskConfig.getTaskDir(task.getId()); + final File attemptDir = new File(taskDir, attemptUUID); + + final TaskLocation taskLocation = TaskLocation.create( + node.getHost(), + node.getPlaintextPort(), + node.getTlsPort() + ); + + try { + final Closer closer = Closer.create(); + try { + if (!attemptDir.mkdirs()) { + throw new IOE("Could not create directories: %s", attemptDir); + } + + final File taskFile = new File(taskDir, "task.json"); + final File reportsFile = new File(attemptDir, "report.json"); + taskReportFileWriter.add(task.getId(), reportsFile); + + final ThreadingTaskRunnerWorkItem taskWorkItem; + // time to adjust process holders + synchronized (tasks) { + taskWorkItem = tasks.get(task.getId()); + + if (taskWorkItem.shutdown) { + throw new IllegalStateException("Task has been shut down!"); + } + + if (taskWorkItem == null) { + log.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); + throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); + } + } + + if (!taskFile.exists()) { + jsonMapper.writeValue(taskFile, task); + } + + // This will block for a while. So we append the thread information with more details + final String priorThreadName = Thread.currentThread().getName(); + Thread.currentThread().setName(StringUtils.format("%s-[%s]", priorThreadName, task.getId())); + + TaskStatus taskStatus = null; + final TaskToolbox toolbox = toolboxFactory.build(task); + try { + ListenableFuture taskStatusFuture = taskExecutor.submit( + new Callable() + { + @Override + public TaskStatus call() + { + taskWorkItem.setThread(Thread.currentThread()); + try { + return task.run(toolbox); + } + catch (Exception e) { + log.error(e, "Task[%s] exited with exception.", task.getId()); + return null; + } + finally { + taskWorkItem.setThread(null); + } + } + } + ); + TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation); + TaskRunnerUtils.notifyStatusChanged( + listeners, + task.getId(), + TaskStatus.running(task.getId()) + ); + taskStatus = taskStatusFuture.get(); + } + finally { + taskWorkItem.setFinished(true); + Thread.currentThread().setName(priorThreadName); + if (reportsFile.exists()) { + taskLogPusher.pushTaskReports(task.getId(), reportsFile); + } + if (taskStatus == null) { + taskStatus = TaskStatus.failure(task.getId()); + } + } + + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), taskStatus); + return taskStatus; + } + catch (Throwable t) { + throw closer.rethrow(t); + } + finally { + closer.close(); + } + } + catch (Throwable t) { + log.info(t, "Exception caught during execution"); + throw new RuntimeException(t); + } + finally { + taskReportFileWriter.delete(task.getId()); + appenderatorsManager.removeAppenderatorForTask(task.getId()); + + try { + synchronized (tasks) { + final ThreadingTaskRunnerWorkItem taskWorkItem = tasks.remove(task.getId()); + if (!stopping) { + saveRunningTasks(); + } + } + + try { + if (!stopping && taskDir.exists()) { + log.info("Removing task directory: %s", taskDir); + FileUtils.deleteDirectory(taskDir); + } + } + catch (Exception e) { + log.makeAlert(e, "Failed to delete task directory") + .addData("taskDir", taskDir.toString()) + .addData("task", task.getId()) + .emit(); + } + } + catch (Exception e) { + log.error(e, "Suppressing exception caught while cleaning up task"); + } + } + } + } + ) + ) + ); + saveRunningTasks(); + return tasks.get(task.getId()).getResult(); + } + } + + @Override + public void shutdown(String taskid, String reason) + { + log.info("Shutdown [%s] because: [%s]", taskid, reason); + final ThreadingTaskRunnerWorkItem taskInfo; + + synchronized (tasks) { + taskInfo = tasks.get(taskid); + + if (taskInfo == null) { + log.info("Ignoring request to cancel unknown task: %s", taskid); + return; + } + + if (taskInfo.shutdown) { + log.info( + "Task [%s] is already shutting down, ignoring duplicate shutdown request with reason [%s]", + taskid, + reason + ); + } else { + taskInfo.shutdown = true; + scheduleTaskShutdown(taskInfo); + } + } + } + + /** + * Submits a callable to the control thread pool that attempts a task graceful shutdown, + * if shutdown is not already scheduled. + * + * The shutdown will wait for the configured timeout and then interrupt the thread if the timeout is exceeded. + */ + private ListenableFuture scheduleTaskShutdown(ThreadingTaskRunnerWorkItem taskInfo) + { + synchronized (tasks) { + if (taskInfo.shutdownFuture != null) { + return taskInfo.shutdownFuture; + } + + ListenableFuture shutdownFuture = controlThreadExecutor.submit( + new Callable() + { + @Override + public Void call() + { + log.info("Stopping thread for task: %s", taskInfo.getTaskId()); + taskInfo.getTask().stopGracefully(taskConfig); + + try { + TaskStatus status = taskInfo.getResult().get( + taskConfig.getGracefulShutdownTimeout().toStandardDuration().getMillis(), + TimeUnit.MILLISECONDS + ); + + if (status == null) { + if (taskInfo.thread != null) { + taskInfo.thread.interrupt(); + } + } + } + catch (Exception e) { + log.info(e, "Encountered exception while waiting for task [%s] shutdown", taskInfo.getTaskId()); + if (taskInfo.thread != null) { + taskInfo.thread.interrupt(); + } + } + return null; + } + } + ); + + taskInfo.shutdownFuture = shutdownFuture; + return shutdownFuture; + } + } + + /** + * First shuts down the task execution pool and then schedules a graceful shutdown attempt for each active task. + * + * After the tasks shutdown gracefully or the graceful shutdown timeout is exceeded, the control thread pool + * will be terminated (also waiting for the graceful shutdown period for this termination). + */ + @Override + public void stop() + { + stopping = true; + taskExecutor.shutdown(); + + List> shutdownFutures = new ArrayList<>(); + synchronized (tasks) { + for (ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) { + shutdownFutures.add(scheduleTaskShutdown(taskWorkItem)); + } + } + controlThreadExecutor.shutdown(); + try { + ListenableFuture> shutdownFuture = Futures.successfulAsList(shutdownFutures); + shutdownFuture.get(); + } + catch (Exception e) { + log.error(e, "Encountered exception when stopping all tasks."); + } + + final DateTime start = DateTimes.nowUtc(); + final long gracefulShutdownMillis = taskConfig.getGracefulShutdownTimeout().toStandardDuration().getMillis(); + + log.info("Waiting up to %,dms for shutdown.", gracefulShutdownMillis); + if (gracefulShutdownMillis > 0) { + try { + final boolean terminated = controlThreadExecutor.awaitTermination( + gracefulShutdownMillis, + TimeUnit.MILLISECONDS + ); + final long elapsed = System.currentTimeMillis() - start.getMillis(); + if (terminated) { + log.info("Finished stopping in %,dms.", elapsed); + } else { + final Set stillRunning; + synchronized (tasks) { + stillRunning = ImmutableSet.copyOf(tasks.keySet()); + } + log.makeAlert("Failed to stop task threads") + .addData("stillRunning", stillRunning) + .addData("elapsed", elapsed) + .emit(); + + log.warn( + "Executor failed to stop after %,dms, not waiting for it! Tasks still running: [%s]", + elapsed, + Joiner.on("; ").join(stillRunning) + ); + } + } + catch (InterruptedException e) { + log.warn(e, "Interrupted while waiting for executor to finish."); + Thread.currentThread().interrupt(); + } + } else { + log.warn("Ran out of time, not waiting for executor to finish!"); + } + } + + @Override + public Collection getRunningTasks() + { + synchronized (tasks) { + final List ret = new ArrayList<>(); + for (final ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) { + if (taskWorkItem.getThread() != null) { + ret.add(taskWorkItem); + } + } + return ret; + } + } + + @Override + public Collection getPendingTasks() + { + synchronized (tasks) { + final List ret = new ArrayList<>(); + for (final ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) { + if (taskWorkItem.getThread() == null) { + ret.add(taskWorkItem); + } + } + return ret; + } + } + + @Override + public Collection getKnownTasks() + { + synchronized (tasks) { + return Lists.newArrayList(tasks.values()); + } + } + + @Nullable + @Override + public RunnerTaskState getRunnerTaskState(String taskId) + { + final ThreadingTaskRunnerWorkItem workItem = tasks.get(taskId); + if (workItem == null) { + return null; + } else { + if (workItem.getThread() == null) { + return RunnerTaskState.PENDING; + } else if (!workItem.isFinished()) { + return RunnerTaskState.RUNNING; + } else { + return RunnerTaskState.NONE; + } + } + } + + @Override + public Optional getScalingStats() + { + return Optional.absent(); + } + + + // Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that + // occur while saving. + @GuardedBy("tasks") + private void saveRunningTasks() + { + final File restoreFile = getRestoreFile(); + final List theTasks = new ArrayList<>(); + for (ThreadingTaskRunnerWorkItem threadingTaskRunnerWorkItem : tasks.values()) { + theTasks.add(threadingTaskRunnerWorkItem.getTaskId()); + } + + try { + Files.createParentDirs(restoreFile); + jsonMapper.writeValue(restoreFile, new TaskRestoreInfo(theTasks)); + } + catch (Exception e) { + log.warn(e, "Failed to save tasks to restore file[%s]. Skipping this save.", restoreFile); + } + } + + private File getRestoreFile() + { + return new File(taskConfig.getBaseTaskDir(), TASK_RESTORE_FILENAME); + } + + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, + Iterable intervals + ) + { + return appenderatorsManager.getQueryRunnerForIntervals(query, intervals); + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, + Iterable specs + ) + { + return appenderatorsManager.getQueryRunnerForSegments(query, specs); + } + + private static class TaskRestoreInfo + { + @JsonProperty + private final List runningTasks; + + @JsonCreator + public TaskRestoreInfo( + @JsonProperty("runningTasks") List runningTasks + ) + { + this.runningTasks = runningTasks; + } + + public List getRunningTasks() + { + return runningTasks; + } + } + + private static class ThreadingTaskRunnerWorkItem extends TaskRunnerWorkItem + { + private final Task task; + private volatile Thread thread; + private boolean finished = false; + private volatile boolean shutdown = false; + private ListenableFuture shutdownFuture; + + private ThreadingTaskRunnerWorkItem( + Task task, + ListenableFuture statusFuture + ) + { + super(task.getId(), statusFuture); + this.task = task; + } + + public Task getTask() + { + return task; + } + + @Override + public TaskLocation getLocation() + { + return null; + } + + @Override + public String getTaskType() + { + return task.getType(); + } + + @Override + public String getDataSource() + { + return task.getDataSource(); + } + + public void setThread(Thread thread) + { + this.thread = thread; + } + + public Thread getThread() + { + return thread; + } + + public boolean isFinished() + { + return finished; + } + + public void setFinished(boolean finished) + { + this.finished = finished; + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java index 532b2fc91548..0545a6e3aa9f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java @@ -463,6 +463,32 @@ public void nodeViewInitialized() } ); + DruidNodeDiscovery druidNodeDiscoveryIndexer = druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER); + druidNodeDiscoveryIndexer.registerListener( + new DruidNodeDiscovery.Listener() + { + @Override + public void nodesAdded(Collection nodes) + { + nodes.forEach(node -> addWorker(toWorker(node))); + } + + @Override + public void nodesRemoved(Collection nodes) + { + nodes.forEach(node -> removeWorker(toWorker(node))); + } + + @Override + public void nodeViewInitialized() + { + //CountDownLatch.countDown() does nothing when count has already reached 0. + workerViewInitialized.countDown(); + } + } + ); + + long workerDiscoveryStartTime = System.currentTimeMillis(); while (!workerViewInitialized.await(30, TimeUnit.SECONDS)) { if (System.currentTimeMillis() - workerDiscoveryStartTime > TimeUnit.MINUTES.toMillis(5)) { 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 11ed7d4d7649..41b891ffb9d8 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 @@ -50,7 +50,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; -import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -75,6 +75,7 @@ public abstract class SeekableStreamIndexTask savedParseExceptions; + protected final AppenderatorsManager appenderatorsManager; // Lazily initialized, to avoid calling it on the overlord when tasks are instantiated. // See https://github.com/apache/incubator-druid/issues/7724 for issues that can cause. @@ -91,7 +92,8 @@ public SeekableStreamIndexTask( @Nullable final ChatHandlerProvider chatHandlerProvider, final AuthorizerMapper authorizerMapper, final RowIngestionMetersFactory rowIngestionMetersFactory, - @Nullable final String groupId + @Nullable final String groupId, + AppenderatorsManager appenderatorsManager ) { super( @@ -114,6 +116,7 @@ public SeekableStreamIndexTask( this.authorizerMapper = authorizerMapper; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.runnerSupplier = Suppliers.memoize(this::createTaskRunner); + this.appenderatorsManager = appenderatorsManager; } private static String makeTaskId(String dataSource, String type) @@ -195,7 +198,8 @@ public QueryRunner getQueryRunner(Query query) public Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox) { - return Appenderators.createRealtime( + return appenderatorsManager.createRealtimeAppenderatorForTask( + getId(), dataSchema, tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, 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 f7de882fda15..07406eaf0397 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 @@ -71,6 +71,7 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -195,6 +196,7 @@ public enum Status private final CircularBuffer savedParseExceptions; private final String stream; private final RowIngestionMeters rowIngestionMeters; + private final AppenderatorsManager appenderatorsManager; private final Set publishingSequences = Sets.newConcurrentHashSet(); private final List> publishWaitList = new ArrayList<>(); @@ -220,7 +222,8 @@ public SeekableStreamIndexTaskRunner( final AuthorizerMapper authorizerMapper, final Optional chatHandlerProvider, final CircularBuffer savedParseExceptions, - final RowIngestionMetersFactory rowIngestionMetersFactory + final RowIngestionMetersFactory rowIngestionMetersFactory, + final AppenderatorsManager appenderatorsManager ) { Preconditions.checkNotNull(task); @@ -233,6 +236,7 @@ public SeekableStreamIndexTaskRunner( this.savedParseExceptions = savedParseExceptions; this.stream = ioConfig.getStartSequenceNumbers().getStream(); this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters(); + this.appenderatorsManager = appenderatorsManager; this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); this.sequences = new CopyOnWriteArrayList<>(); this.ingestionState = IngestionState.NOT_STARTED; @@ -248,7 +252,7 @@ public TaskStatus run(TaskToolbox toolbox) catch (Exception e) { log.error(e, "Encountered exception while running task."); final String errorMsg = Throwables.getStackTraceAsString(e); - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(errorMsg)); + toolbox.getTaskReportFileWriter().write(task.getId(), getTaskCompletionReports(errorMsg)); return TaskStatus.failure( task.getId(), errorMsg @@ -376,9 +380,11 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception Throwable caughtExceptionOuter = null; try (final RecordSupplier recordSupplier = task.newTaskRecordSupplier()) { - toolbox.getDataSegmentServerAnnouncer().announce(); - toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + if (appenderatorsManager.shouldTaskMakeNodeAnnouncements()) { + toolbox.getDataSegmentServerAnnouncer().announce(); + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + } appenderator = task.newAppenderator(fireDepartmentMetrics, toolbox); driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics); @@ -838,8 +844,10 @@ public void onFailure(Throwable t) chatHandlerProvider.get().unregister(task.getId()); } - toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); - toolbox.getDataSegmentServerAnnouncer().unannounce(); + if (appenderatorsManager.shouldTaskMakeNodeAnnouncements()) { + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); + toolbox.getDataSegmentServerAnnouncer().unannounce(); + } } catch (Throwable e) { if (caughtExceptionOuter != null) { @@ -850,7 +858,7 @@ public void onFailure(Throwable t) } } - toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(null)); + toolbox.getTaskReportFileWriter().write(task.getId(), getTaskCompletionReports(null)); return TaskStatus.success(task.getId()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java index ac5c15c09fac..3f22dad186a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/WorkerTaskManager.java @@ -158,6 +158,8 @@ public void stop() throws Exception synchronized (lock) { try { + // When stopping, the task status should not be communicated to the overlord, so the listener and exec + // are shut down before the taskRunner is stopped. taskRunner.unregisterListener("WorkerTaskManager"); exec.shutdownNow(); taskRunner.stop(); @@ -693,7 +695,6 @@ public void handle() changeHistory.addChangeRequest(new WorkerHistoryItem.TaskUpdate(latest)); taskAnnouncementChanged(latest); - log.info( "Job's finished. Completed [%s] with status [%s]", task.getId(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java index 1b0430a05460..33011f952ef9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java @@ -28,6 +28,7 @@ import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; @@ -38,6 +39,7 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -88,6 +90,7 @@ public int columnCacheSizeBytes() .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) .addValue(IndexingServiceClient.class, new NoopIndexingServiceClient()) .addValue(AuthorizerMapper.class, new AuthorizerMapper(ImmutableMap.of())) + .addValue(AppenderatorsManager.class, new TestAppenderatorsManager()) .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index c1d682e1e280..c72fea9a096b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -51,8 +51,8 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.SegmentLoaderFactory; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; @@ -119,6 +119,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.StorageLocationConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.ExpressionTransform; @@ -274,6 +275,7 @@ public Firehose connect(InputRowParser parser, File temporaryDirectory) throws P private File baseDir; private File reportsFile; private RowIngestionMetersFactory rowIngestionMetersFactory; + private AppenderatorsManager appenderatorsManager; @Before public void setUp() throws IOException @@ -289,6 +291,8 @@ public void setUp() throws IOException derbyConnector.createSegmentTable(); derbyConnector.createPendingSegmentsTable(); + appenderatorsManager = new TestAppenderatorsManager(); + baseDir = tempFolder.newFolder(); reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json"); makeToolboxFactory(baseDir); @@ -1431,7 +1435,8 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ) { @Override @@ -1620,7 +1625,7 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new TaskReportFileWriter(reportsFile) + new SingleFileTaskReportFileWriter(reportsFile) ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index f157513d7092..f1400be1a089 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -53,6 +53,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; import org.apache.druid.segment.loading.StorageLocationConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -104,6 +105,7 @@ public class CompactionTaskRunTest extends IngestionTestBase private CoordinatorClient coordinatorClient; private SegmentLoaderFactory segmentLoaderFactory; private ExecutorService exec; + private AppenderatorsManager appenderatorsManager; private static RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig()); public CompactionTaskRunTest() @@ -119,6 +121,7 @@ public List getDatabaseSegmentDataSourceSegments(String dataSource, } }; segmentLoaderFactory = new SegmentLoaderFactory(getIndexIO(), getObjectMapper()); + appenderatorsManager = new TestAppenderatorsManager(); } @Before @@ -146,7 +149,8 @@ public void testRun() throws Exception rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); final CompactionTask compactionTask = builder @@ -179,7 +183,8 @@ public void testRunCompactionTwice() throws Exception rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); final CompactionTask compactionTask1 = builder @@ -228,7 +233,8 @@ public void testWithSegmentGranularity() throws Exception rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); // day segmentGranularity @@ -301,7 +307,8 @@ private Pair> runIndexTask() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); return runTask(indexTask); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 69f10f6be400..c7ce6d080b73 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -96,6 +96,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; @@ -154,6 +155,7 @@ public class CompactionTaskTest private static RowIngestionMetersFactory rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); private static Map segmentMap = new HashMap<>(); private static CoordinatorClient coordinatorClient = new TestCoordinatorClient(segmentMap); + private static AppenderatorsManager appenderatorsManager = new TestAppenderatorsManager(); private static ObjectMapper objectMapper = setupInjectablesInObjectMapper(new DefaultObjectMapper()); private static RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig()); @@ -244,6 +246,7 @@ private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMa binder.bind(RowIngestionMetersFactory.class).toInstance(rowIngestionMetersFactory); binder.bind(CoordinatorClient.class).toInstance(coordinatorClient); binder.bind(SegmentLoaderFactory.class).toInstance(new SegmentLoaderFactory(null, objectMapper)); + binder.bind(AppenderatorsManager.class).toInstance(appenderatorsManager); } ) ) @@ -327,7 +330,8 @@ public void testSerdeWithInterval() throws IOException rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); final CompactionTask task = builder .interval(COMPACTION_INTERVAL) @@ -351,7 +355,8 @@ public void testSerdeWithSegments() throws IOException rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); final CompactionTask task = builder .segments(SEGMENTS) @@ -375,7 +380,8 @@ public void testSerdeWithDimensions() throws IOException rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); final CompactionTask task = builder @@ -820,7 +826,8 @@ public void testEmptyInterval() rowIngestionMetersFactory, coordinatorClient, segmentLoaderFactory, - retryPolicyFactory + retryPolicyFactory, + appenderatorsManager ); final CompactionTask task = builder diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 97279f94b643..a8423186c6c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -38,10 +38,10 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LockAcquireAction; @@ -87,6 +87,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; import org.apache.druid.segment.loading.StorageLocationConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; @@ -155,6 +156,7 @@ public class IndexTaskTest private volatile int segmentAllocatePartitionCounter; private File reportsFile; private RowIngestionMetersFactory rowIngestionMetersFactory; + private AppenderatorsManager appenderatorsManager; public IndexTaskTest() { @@ -169,6 +171,7 @@ public IndexTaskTest() @Before public void setup() throws IOException { + appenderatorsManager = new TestAppenderatorsManager(); reportsFile = temporaryFolder.newFile( StringUtils.format("IndexTaskTestReports-%s.json", System.currentTimeMillis()) ); @@ -246,7 +249,8 @@ public void testDeterminePartitions() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -298,7 +302,8 @@ public void testTransformSpec() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); Assert.assertEquals(indexTask.getId(), indexTask.getGroupId()); @@ -342,7 +347,8 @@ public void testWithArbitraryGranularity() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -379,7 +385,8 @@ public void testIntervalBucketing() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -412,7 +419,8 @@ public void testNumShardsProvided() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -450,7 +458,8 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); runTask(indexTask); @@ -525,7 +534,8 @@ public void testAppendToExisting() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); Assert.assertEquals("index_append_test", indexTask.getGroupId()); @@ -575,7 +585,8 @@ public void testIntervalNotSpecified() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -638,7 +649,8 @@ public void testCSVFileWithHeader() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -690,7 +702,8 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -737,7 +750,8 @@ public void testWithSmallMaxTotalRows() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -782,7 +796,8 @@ public void testPerfectRollup() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -826,7 +841,8 @@ public void testBestEffortRollup() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -904,7 +920,8 @@ public void testIgnoreParseException() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -957,7 +974,8 @@ public void testReportParseException() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); TaskStatus status = runTask(indexTask).lhs; @@ -1047,7 +1065,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); TaskStatus status = runTask(indexTask).lhs; @@ -1171,7 +1190,8 @@ public void testMultipleParseExceptionsFailure() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); TaskStatus status = runTask(indexTask).lhs; @@ -1286,7 +1306,8 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); TaskStatus status = runTask(indexTask).lhs; @@ -1385,7 +1406,8 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); final List segments = runTask(indexTask).rhs; @@ -1455,7 +1477,8 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + appenderatorsManager ); TaskStatus status = runTask(indexTask).lhs; @@ -1578,7 +1601,7 @@ public void killAll() null, null, null, - new TaskReportFileWriter(reportsFile) + new SingleFileTaskReportFileWriter(reportsFile) ); indexTask.isReady(box.getTaskActionClient()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java index 3e7b8fdc3726..0fe9d7fcf1f2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java @@ -19,12 +19,12 @@ package org.apache.druid.indexing.common.task; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskReportFileWriter; import java.util.Map; -public class NoopTestTaskFileWriter extends TaskReportFileWriter +public class NoopTestTaskFileWriter extends SingleFileTaskReportFileWriter { public NoopTestTaskFileWriter() { @@ -32,7 +32,7 @@ public NoopTestTaskFileWriter() } @Override - public void write(Map reports) + public void write(String id, Map reports) { } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index a09dd65d95ab..73163302a3e3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -217,7 +217,8 @@ public void testIndexTaskSerde() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ); final String json = jsonMapper.writeValueAsString(task); @@ -299,7 +300,8 @@ public void testIndexTaskwithResourceSerde() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - rowIngestionMetersFactory + rowIngestionMetersFactory, + null ); for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java new file mode 100644 index 000000000000..6eaaad5171e0 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -0,0 +1,152 @@ +/* + * 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.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.cache.Cache; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.joda.time.Interval; + +import java.util.concurrent.ExecutorService; + +public class TestAppenderatorsManager implements AppenderatorsManager +{ + private Appenderator realtimeAppenderator; + + public TestAppenderatorsManager() + { + + } + + @Override + public Appenderator createRealtimeAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats + ) + { + realtimeAppenderator = Appenderators.createRealtime( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + conglomerate, + segmentAnnouncer, + emitter, + queryExecutorService, + cache, + cacheConfig, + cachePopulatorStats + ); + return realtimeAppenderator; + } + + @Override + public Appenderator createOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger + ) + { + return Appenderators.createOffline( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger + ); + } + + @Override + public void removeAppenderatorForTask(String taskId) + { + + } + + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, + Iterable intervals + ) + { + if (realtimeAppenderator != null) { + return realtimeAppenderator.getQueryRunnerForIntervals(query, intervals); + } else { + return null; + } + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, + Iterable specs + ) + { + if (realtimeAppenderator != null) { + return realtimeAppenderator.getQueryRunnerForSegments(query, specs); + } else { + return null; + } + } + + @Override + public boolean shouldTaskMakeNodeAnnouncements() + { + return true; + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 49315d3973d0..a9bc7f4786fe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -23,7 +23,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.SegmentLoaderFactory; -import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; @@ -104,7 +104,7 @@ public void setup() throws IOException node, null, null, - new TaskReportFileWriter(new File("fake")) + new SingleFileTaskReportFileWriter(new File("fake")) ); runner = new SingleTaskBackgroundRunner( toolboxFactory, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 6bdfbe4371b9..143d2b5e5f60 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -68,6 +68,7 @@ import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; @@ -108,6 +109,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartmentTest; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; @@ -226,6 +228,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private TaskQueueConfig tqc; private TaskConfig taskConfig; private DataSegmentPusher dataSegmentPusher; + private AppenderatorsManager appenderatorsManager; private int pushedSegments; private int announcedSinks; @@ -527,6 +530,8 @@ private TaskToolboxFactory setUpTaskToolboxFactory( Preconditions.checkNotNull(taskStorage); Preconditions.checkNotNull(emitter); + appenderatorsManager = new TestAppenderatorsManager(); + taskLockbox = new TaskLockbox(taskStorage); tac = new LocalTaskActionClientFactory( taskStorage, @@ -550,6 +555,7 @@ public List getLocations() return new ArrayList<>(); } }; + return new TaskToolboxFactory( taskConfig, tac, @@ -705,7 +711,8 @@ public void testIndexTask() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - ROW_INGESTION_METERS_FACTORY + ROW_INGESTION_METERS_FACTORY, + appenderatorsManager ); final Optional preRunTaskStatus = tsqa.getStatus(indexTask.getId()); @@ -786,7 +793,8 @@ public void testIndexTaskFailure() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - ROW_INGESTION_METERS_FACTORY + ROW_INGESTION_METERS_FACTORY, + null ); final TaskStatus status = runTask(indexTask); @@ -1174,7 +1182,8 @@ public void testResumeTasks() throws Exception null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null, - ROW_INGESTION_METERS_FACTORY + ROW_INGESTION_METERS_FACTORY, + appenderatorsManager ); final long startTime = System.currentTimeMillis(); 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 ae55fffc637e..252c166200e8 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 @@ -705,7 +705,8 @@ public TestSeekableStreamIndexTask( chatHandlerProvider, authorizerMapper, rowIngestionMetersFactory, - groupId + groupId, + null ); } diff --git a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java index 4dcb77759a86..fc1cc8375ebc 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java @@ -40,9 +40,9 @@ public abstract class DruidNodeDiscoveryProvider { private static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( - LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.BROKER, NodeType.HISTORICAL, NodeType.PEON), - DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.HISTORICAL, NodeType.PEON), - WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.PEON) + LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.BROKER, NodeType.HISTORICAL, NodeType.PEON, NodeType.INDEXER), + DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.HISTORICAL, NodeType.PEON, NodeType.INDEXER), + WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.PEON, NodeType.INDEXER) ); private final ConcurrentHashMap serviceDiscoveryMap = diff --git a/server/src/main/java/org/apache/druid/discovery/NodeType.java b/server/src/main/java/org/apache/druid/discovery/NodeType.java index 841656bf1fc9..1f6f90d8a12e 100644 --- a/server/src/main/java/org/apache/druid/discovery/NodeType.java +++ b/server/src/main/java/org/apache/druid/discovery/NodeType.java @@ -36,7 +36,8 @@ public enum NodeType OVERLORD("overlord"), PEON("peon"), ROUTER("router"), - MIDDLE_MANAGER("middleManager"); + MIDDLE_MANAGER("middleManager"), + INDEXER("indexer"); private final String jsonName; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/UnifiedIndexerLifecycleHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/UnifiedIndexerLifecycleHandler.java new file mode 100644 index 000000000000..de28c3ad1d1c --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/UnifiedIndexerLifecycleHandler.java @@ -0,0 +1,99 @@ +/* + * 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.segment.realtime; + +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; + +import java.io.IOException; + +@ManageLifecycle +public class UnifiedIndexerLifecycleHandler +{ + private static final EmittingLogger log = new EmittingLogger(UnifiedIndexerLifecycleHandler.class); + + private final DataSegmentServerAnnouncer dataSegmentServerAnnouncer; + + // Synchronizes start/stop of this object. + private final Object startStopLock = new Object(); + + private volatile boolean started = false; + + @Inject + public UnifiedIndexerLifecycleHandler( + DataSegmentServerAnnouncer dataSegmentServerAnnouncer + ) + { + this.dataSegmentServerAnnouncer = dataSegmentServerAnnouncer; + } + + @LifecycleStart + public void start() throws IOException + { + synchronized (startStopLock) { + if (started) { + return; + } + + log.info("Starting..."); + try { + dataSegmentServerAnnouncer.announce(); + } + catch (Exception e) { + Throwables.propagateIfPossible(e, IOException.class); + throw new RuntimeException(e); + } + started = true; + log.info("Started."); + } + } + + @LifecycleStop + public void stop() + { + synchronized (startStopLock) { + if (!started) { + return; + } + + log.info("Stopping..."); + try { + dataSegmentServerAnnouncer.unannounce(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + started = false; + } + log.info("Stopped."); + } + } + + public boolean isStarted() + { + return started; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 647340bbb502..481482f4a6e4 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -129,9 +129,7 @@ public class AppenderatorImpl implements Appenderator */ private final ConcurrentMap sinks = new ConcurrentHashMap<>(); private final Set droppingSinks = Sets.newConcurrentHashSet(); - private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); + private final VersionedIntervalTimeline sinkTimeline; private final long maxBytesTuningConfig; private final QuerySegmentWalker texasRanger; @@ -183,6 +181,9 @@ public class AppenderatorImpl implements Appenderator this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); this.cache = cache; + this.sinkTimeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); this.texasRanger = conglomerate == null ? null : new SinkQuerySegmentWalker( schema.getDataSource(), sinkTimeline, @@ -198,6 +199,42 @@ public class AppenderatorImpl implements Appenderator log.info("Created Appenderator for dataSource[%s].", schema.getDataSource()); } + /** + * This constructor allows the caller to provide its own sink timeline and segment walker. + * + * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple + * Appenderators. + */ + AppenderatorImpl( + DataSchema schema, + AppenderatorConfig tuningConfig, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + DataSegmentAnnouncer segmentAnnouncer, + VersionedIntervalTimeline sinkTimeline, + SinkQuerySegmentWalker sinkQuerySegmentWalker, + IndexIO indexIO, + IndexMerger indexMerger, + Cache cache + ) + { + this.schema = Preconditions.checkNotNull(schema, "schema"); + this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); + this.metrics = Preconditions.checkNotNull(metrics, "metrics"); + this.dataSegmentPusher = Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher"); + this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); + this.segmentAnnouncer = Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); + this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); + this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); + this.cache = cache; + this.texasRanger = sinkQuerySegmentWalker; + this.sinkTimeline = sinkTimeline; + maxBytesTuningConfig = TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()); + log.info("Created Appenderator for dataSource[%s].", schema.getDataSource()); + } + + @Override public String getDataSource() { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java new file mode 100644 index 000000000000..bec7c1d4aa2b --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -0,0 +1,127 @@ +/* + * 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.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.cache.Cache; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.joda.time.Interval; + +import java.util.concurrent.ExecutorService; + +/** + * This interface defines entities that create and manage potentially multiple Appenderators. + * + * The AppenderatorsManager should be used by a Task running in a Peon or an Indexer process when it needs + * an Appenderator. + * + * The AppenderatorsManager also provides methods for creating QueryRunners that read the data held by the + * Appenderators created through the AppenderatorsManager. + * + * In later updates, this interface will be used to manage memory usage across multiple Appenderators, + * useful for the Indexer where all Tasks run in the same process. + * + * The methods on AppenderatorsManager can be called by multiple threads. + */ +public interface AppenderatorsManager +{ + /** + * Creates an Appenderator suited for realtime ingestion. Note that this method's parameters include objects + * used for query processing. + */ + Appenderator createRealtimeAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats + ); + + /** + * Creates an Appenderator suited for batch ingestion. + */ + Appenderator createOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger + ); + + /** + * Removes any internal Appenderator-tracking state associated with the provided taskId. + * + * This method should be called when a task is finished using its Appenderator that was previously created by + * createRealtimeAppenderatorForTask or createOfflineAppenderatorForTask. + * + * The method can be called by the entity managing Tasks when the Tasks finish, such as ThreadingTaskRunner. + */ + void removeAppenderatorForTask(String taskId); + + /** + * Returns a query runner for the given intervals over the Appenderators managed by this AppenderatorsManager. + */ + QueryRunner getQueryRunnerForIntervals( + Query query, + Iterable intervals + ); + + /** + * Returns a query runner for the given segment specs over the Appenderators managed by this AppenderatorsManager. + */ + QueryRunner getQueryRunnerForSegments( + Query query, + Iterable specs + ); + + /** + * As AppenderatorsManager implementions are service dependent (i.e., Peons and Indexers have different impls), + * this method allows Tasks to know whether they should announce themselves as nodes and segment servers + * to the rest of the cluster. + * + * Only Tasks running in Peons (i.e., as separate processes) should make their own individual node announcements. + */ + boolean shouldTaskMakeNodeAnnouncements(); +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java new file mode 100644 index 000000000000..23b4567a895b --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -0,0 +1,120 @@ +/* + * 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.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.cache.Cache; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.joda.time.Interval; + +import java.util.concurrent.ExecutorService; + +/** + * This implementation is needed because Overlords and MiddleManagers operate on Task objects which + * can require an AppenderatorsManager to be injected. + * + * The methods of this implementation throw exceptions because the Overlord/MM should never be calling + * the AppenderatorsManager. + */ +public class DummyForInjectionAppenderatorsManager implements AppenderatorsManager +{ + private static final String ERROR_MSG = + "AppenderatorsManager methods should only called by services that run tasks directly."; + + @Override + public Appenderator createRealtimeAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats + ) + { + throw new UOE(ERROR_MSG); + } + + @Override + public Appenderator createOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger + ) + { + throw new UOE(ERROR_MSG); + } + + @Override + public void removeAppenderatorForTask(String taskId) + { + throw new UOE(ERROR_MSG); + } + + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, + Iterable intervals + ) + { + throw new UOE(ERROR_MSG); + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, + Iterable specs + ) + { + throw new UOE(ERROR_MSG); + } + + @Override + public boolean shouldTaskMakeNodeAnnouncements() + { + throw new UOE(ERROR_MSG); + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java new file mode 100644 index 000000000000..25324c80c8f9 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -0,0 +1,158 @@ +/* + * 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.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.cache.Cache; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.joda.time.Interval; + +import java.util.concurrent.ExecutorService; + +public class PeonAppenderatorsManager implements AppenderatorsManager +{ + private Appenderator realtimeAppenderator; + private Appenderator batchAppenderator; + + @Override + public Appenderator createRealtimeAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats + ) + { + if (realtimeAppenderator != null) { + throw new ISE("A realtime appenderator was already created for this peon's task."); + } else if (batchAppenderator != null) { + throw new ISE("A batch appenderator was already created for this peon's task."); + } else { + realtimeAppenderator = Appenderators.createRealtime( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger, + conglomerate, + segmentAnnouncer, + emitter, + queryExecutorService, + cache, + cacheConfig, + cachePopulatorStats + ); + } + return realtimeAppenderator; + } + + @Override + public Appenderator createOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger + ) + { + if (realtimeAppenderator != null) { + throw new ISE("A realtime appenderator was already created for this peon's task."); + } else if (batchAppenderator != null) { + throw new ISE("A batch appenderator was already created for this peon's task."); + } else { + batchAppenderator = Appenderators.createOffline( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger + ); + return batchAppenderator; + } + } + + @Override + public void removeAppenderatorForTask(String taskId) + { + // only one appenderator and task, and the process will shutdown later, don't need to do anything + } + + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, + Iterable intervals + ) + { + if (realtimeAppenderator == null) { + throw new ISE("Was asked for a query runner but realtimeAppenderator was null!"); + } else { + return realtimeAppenderator.getQueryRunnerForIntervals(query, intervals); + } + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, + Iterable specs + ) + { + if (realtimeAppenderator == null) { + throw new ISE("Was asked for a query runner but realtimeAppenderator was null!"); + } else { + return realtimeAppenderator.getQueryRunnerForSegments(query, specs); + } + } + + @Override + public boolean shouldTaskMakeNodeAnnouncements() + { + return true; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 7d08c0f7849e..69cef4c64df6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -74,6 +74,7 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker private static final String CONTEXT_SKIP_INCREMENTAL_SEGMENT = "skipIncrementalSegment"; private final String dataSource; + private final VersionedIntervalTimeline sinkTimeline; private final ObjectMapper objectMapper; private final ServiceEmitter emitter; @@ -312,6 +313,11 @@ private QueryRunner withPerSinkMetrics( ); } + public VersionedIntervalTimeline getSinkTimeline() + { + return sinkTimeline; + } + public static String makeHydrantCacheIdentifier(FireHydrant input) { return input.getSegmentId() + "_" + input.getCount(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java new file mode 100644 index 000000000000..64c45ba8671c --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -0,0 +1,206 @@ +/* + * 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.segment.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import org.apache.druid.client.cache.Cache; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.guice.annotations.Processing; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.plumber.Sink; +import org.apache.druid.server.coordination.DataSegmentAnnouncer; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.joda.time.Interval; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; + +/** + * Manages Appenderators for the Indexer task execution service, which runs all tasks in a single process. + * + * This class keeps two maps: + * - A per-datasource SinkQuerySegmentWalker (with an associated per-datasource timeline) + * - A map that associates a taskId with the Appenderator created for that task + * + * Appenderators created by this class will use the shared per-datasource SinkQuerySegmentWalkers. + * + * The per-datasource SinkQuerySegmentWalkers share a common queryExecutorService. + */ +public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager +{ + private static final Logger log = new Logger(UnifiedIndexerAppenderatorsManager.class); + + private final ConcurrentHashMap datasourceSegmentWalkers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap taskAppenderatorMap = new ConcurrentHashMap<>(); + + private final ExecutorService queryExecutorService; + private final Cache cache; + private final CacheConfig cacheConfig; + private final CachePopulatorStats cachePopulatorStats; + + @Inject + public UnifiedIndexerAppenderatorsManager( + @Processing ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats + ) + { + this.queryExecutorService = queryExecutorService; + this.cache = cache; + this.cacheConfig = cacheConfig; + this.cachePopulatorStats = cachePopulatorStats; + } + + @Override + public Appenderator createRealtimeAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ServiceEmitter emitter, + ExecutorService queryExecutorService, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats + ) + { + SinkQuerySegmentWalker segmentWalker = datasourceSegmentWalkers.computeIfAbsent( + schema.getDataSource(), + (datasource) -> { + VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); + SinkQuerySegmentWalker datasourceSegmentWalker = new SinkQuerySegmentWalker( + schema.getDataSource(), + sinkTimeline, + objectMapper, + emitter, + conglomerate, + this.queryExecutorService, + Preconditions.checkNotNull(this.cache, "cache"), + this.cacheConfig, + this.cachePopulatorStats + ); + return datasourceSegmentWalker; + } + ); + + Appenderator appenderator = new AppenderatorImpl( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + segmentAnnouncer, + segmentWalker.getSinkTimeline(), + segmentWalker, + indexIO, + indexMerger, + cache + ); + + taskAppenderatorMap.put(taskId, appenderator); + return appenderator; + } + + @Override + public Appenderator createOfflineAppenderatorForTask( + String taskId, + DataSchema schema, + AppenderatorConfig config, + FireDepartmentMetrics metrics, + DataSegmentPusher dataSegmentPusher, + ObjectMapper objectMapper, + IndexIO indexIO, + IndexMerger indexMerger + ) + { + Appenderator appenderator = Appenderators.createOffline( + schema, + config, + metrics, + dataSegmentPusher, + objectMapper, + indexIO, + indexMerger + ); + taskAppenderatorMap.put(taskId, appenderator); + return appenderator; + } + + @Override + public void removeAppenderatorForTask(String taskId) + { + taskAppenderatorMap.remove(taskId); + } + + @Override + public QueryRunner getQueryRunnerForIntervals( + Query query, + Iterable intervals + ) + { + SinkQuerySegmentWalker segmentWalker = datasourceSegmentWalkers.get(query.getDataSource().toString()); + if (segmentWalker == null) { + throw new IAE("Could not find segment walker for datasource [%s]", query.getDataSource().toString()); + } + return segmentWalker.getQueryRunnerForIntervals(query, intervals); + } + + @Override + public QueryRunner getQueryRunnerForSegments( + Query query, + Iterable specs + ) + { + SinkQuerySegmentWalker segmentWalker = datasourceSegmentWalkers.get(query.getDataSource().toString()); + if (segmentWalker == null) { + throw new IAE("Could not find segment walker for datasource [%s]", query.getDataSource().toString()); + } + return segmentWalker.getQueryRunnerForSegments(query, specs); + } + + @Override + public boolean shouldTaskMakeNodeAnnouncements() + { + return false; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java index 98f508f0bfa2..60a73153471c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/ChatHandlerResource.java @@ -32,7 +32,7 @@ import javax.ws.rs.core.HttpHeaders; import java.util.List; -@Path("/druid/worker/v1") +@Path("/druid/worker/v1/chat") public class ChatHandlerResource { public static final String TASK_ID_HEADER = "X-Druid-Task-Id"; @@ -47,7 +47,7 @@ public ChatHandlerResource(final ChatHandlerProvider handlers, final DataSourceT this.taskId = taskIdHolder.getTaskId(); } - @Path("/chat/{id}") + @Path("/{id}") public Object doTaskChat(@PathParam("id") String handlerId, @Context HttpHeaders headers) { if (taskId != null) { diff --git a/server/src/main/java/org/apache/druid/server/http/ClusterResource.java b/server/src/main/java/org/apache/druid/server/http/ClusterResource.java index 0bc871386c1e..926b90fb2ee7 100644 --- a/server/src/main/java/org/apache/druid/server/http/ClusterResource.java +++ b/server/src/main/java/org/apache/druid/server/http/ClusterResource.java @@ -74,6 +74,11 @@ public Response getClusterServers(@QueryParam("full") boolean full) entityBuilder.put(NodeType.MIDDLE_MANAGER, mmNodes); } + Collection indexerNodes = getNodes(NodeType.INDEXER, full); + if (!indexerNodes.isEmpty()) { + entityBuilder.put(NodeType.INDEXER, indexerNodes); + } + Collection routerNodes = getNodes(NodeType.ROUTER, full); if (!routerNodes.isEmpty()) { entityBuilder.put(NodeType.ROUTER, routerNodes); diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java index 6eafec811402..bb8ea39afc44 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java @@ -48,10 +48,12 @@ public class ChatHandlerServerModule implements Module { private static final String MAX_CHAT_REQUESTS_PROPERTY = "druid.indexer.server.maxChatRequests"; private final Properties properties; + private final boolean useSeparatePort; - public ChatHandlerServerModule(Properties properties) + public ChatHandlerServerModule(Properties properties, boolean useSeparatePort) { this.properties = properties; + this.useSeparatePort = useSeparatePort; } @Override @@ -67,11 +69,21 @@ public void configure(Binder binder) Multibinder.newSetBinder(binder, ServletFilterHolder.class).addBinding().to(TaskIdResponseHeaderFilterHolder.class); - /** - * We bind {@link DruidNode} annotated with {@link RemoteChatHandler} to {@literal @}{@link Self} {@link DruidNode} - * so that same Jetty Server is used for querying as well as ingestion. - */ - binder.bind(DruidNode.class).annotatedWith(RemoteChatHandler.class).to(Key.get(DruidNode.class, Self.class)); + if (useSeparatePort) { + // bind a modified DruidNode that will be used by the Jetty server installed below + binder.bind(DruidNode.class) + .annotatedWith(RemoteChatHandler.class) + .toInstance(makeDruidNodeForSeparateChatHandler()); + + // this installs a separate Jetty server for chat handling + LifecycleModule.register(binder, Server.class, RemoteChatHandler.class); + } else { + /** + * We bind {@link DruidNode} annotated with {@link RemoteChatHandler} to {@literal @}{@link Self} {@link DruidNode} + * so that same Jetty Server is used for querying as well as ingestion. + */ + binder.bind(DruidNode.class).annotatedWith(RemoteChatHandler.class).to(Key.get(DruidNode.class, Self.class)); + } binder.bind(ServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(ServerConfig.class)); binder.bind(TLSServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(TLSServerConfig.class)); } @@ -106,4 +118,33 @@ public Server getServer( injector.getInstance(TLSCertificateChecker.class) ); } + + + /** + * @return Creates a DruidNode identical to the @Self DruidNode but with the port numbers incremented by 1. + */ + private DruidNode makeDruidNodeForSeparateChatHandler() + { + String serviceName = properties.getProperty("druid.service"); + String host = properties.getProperty("druid.host"); + String bindOnHost = properties.getProperty("druid.bindOnHost", "false"); + String plaintextPort = properties.getProperty("druid.plaintextPort"); + String port = properties.getProperty("druid.port"); + String tlsPort = properties.getProperty("druid.tlsPort"); + String enablePlaintextPort = properties.getProperty("druid.enablePlaintextPort", "true"); + String enableTlsPort = properties.getProperty("druid.enableTlsPort", "false"); + + DruidNode chatHandlerNode = new DruidNode( + serviceName, + host, + Boolean.parseBoolean(bindOnHost), + plaintextPort == null ? null : Integer.parseInt(plaintextPort), + port == null ? null : Integer.parseInt(port) + 1, + tlsPort == null ? null : Integer.parseInt(tlsPort) + 1, + Boolean.parseBoolean(enablePlaintextPort), + Boolean.parseBoolean(enableTlsPort) + ); + + return chatHandlerNode; + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java new file mode 100644 index 000000000000..20207264424c --- /dev/null +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -0,0 +1,306 @@ +/* + * 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.cli; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.MapBinder; +import com.google.inject.name.Names; +import io.airlift.airline.Command; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.indexing.HttpIndexingServiceClient; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.discovery.LookupNodeService; +import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.WorkerNodeService; +import org.apache.druid.guice.Binders; +import org.apache.druid.guice.CacheModule; +import org.apache.druid.guice.DruidProcessingModule; +import org.apache.druid.guice.IndexingServiceFirehoseModule; +import org.apache.druid.guice.IndexingServiceModuleHelper; +import org.apache.druid.guice.IndexingServiceTaskLogsModule; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.NodeTypeConfig; +import org.apache.druid.guice.PolyBind; +import org.apache.druid.guice.QueryRunnerFactoryModule; +import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.QueryablePeonModule; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.RetryPolicyConfig; +import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; +import org.apache.druid.indexing.common.actions.RemoteTaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TaskActionToolbox; +import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClient; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClientFactory; +import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.ThreadingTaskRunner; +import org.apache.druid.indexing.worker.Worker; +import org.apache.druid.indexing.worker.WorkerCuratorCoordinator; +import org.apache.druid.indexing.worker.WorkerTaskMonitor; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.indexing.worker.http.TaskManagementResource; +import org.apache.druid.indexing.worker.http.WorkerResource; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.lookup.LookupModule; +import org.apache.druid.segment.loading.DataSegmentArchiver; +import org.apache.druid.segment.loading.DataSegmentKiller; +import org.apache.druid.segment.loading.DataSegmentMover; +import org.apache.druid.segment.loading.OmniDataSegmentArchiver; +import org.apache.druid.segment.loading.OmniDataSegmentKiller; +import org.apache.druid.segment.loading.OmniDataSegmentMover; +import org.apache.druid.segment.realtime.UnifiedIndexerLifecycleHandler; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; +import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; +import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; +import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig; +import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory; +import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.http.SegmentListerResource; +import org.apache.druid.server.initialization.jetty.ChatHandlerServerModule; +import org.apache.druid.server.initialization.jetty.JettyServerInitializer; +import org.eclipse.jetty.server.Server; + +import java.util.List; +import java.util.Properties; + +/** + * + */ +@Command( + name = "indexer", + description = "Runs an Indexer. Description TBD." +) +public class CliIndexer extends ServerRunnable +{ + private static final Logger log = new Logger(CliIndexer.class); + + @Inject + private Properties properties; + + public CliIndexer() + { + super(log); + } + + @Override + protected List getModules() + { + return ImmutableList.of( + new DruidProcessingModule(), + new QueryableModule(), + new QueryRunnerFactoryModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/indexer"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8091); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8291); + + IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); + + JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); + JsonConfigProvider.bind(binder, "druid.worker", WorkerConfig.class); + JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.class); + JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); + + binder.bind(TaskReportFileWriter.class).toInstance(new TaskReportFileWriter()); + + binder.bind(TaskRunner.class).to(ThreadingTaskRunner.class); + binder.bind(QuerySegmentWalker.class).to(ThreadingTaskRunner.class); + binder.bind(ThreadingTaskRunner.class).in(LazySingleton.class); + + binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); + + configureTaskActionClient(binder); + binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); + binder.bind(new TypeLiteral>(){}) + .to(ParallelIndexTaskClientFactory.class) + .in(LazySingleton.class); + + PolyBind.createChoice( + binder, + "druid.indexer.task.rowIngestionMeters.type", + Key.get(RowIngestionMetersFactory.class), + Key.get(DropwizardRowIngestionMetersFactory.class) + ); + final MapBinder rowIngestionMetersHandlerProviderBinder = + PolyBind.optionBinder(binder, Key.get(RowIngestionMetersFactory.class)); + rowIngestionMetersHandlerProviderBinder + .addBinding("dropwizard") + .to(DropwizardRowIngestionMetersFactory.class) + .in(LazySingleton.class); + binder.bind(DropwizardRowIngestionMetersFactory.class).in(LazySingleton.class); + + PolyBind.createChoice( + binder, + "druid.indexer.task.chathandler.type", + Key.get(ChatHandlerProvider.class), + Key.get(ServiceAnnouncingChatHandlerProvider.class) + ); + final MapBinder handlerProviderBinder = + PolyBind.optionBinder(binder, Key.get(ChatHandlerProvider.class)); + handlerProviderBinder + .addBinding("announce") + .to(ServiceAnnouncingChatHandlerProvider.class) + .in(LazySingleton.class); + handlerProviderBinder + .addBinding("noop") + .to(NoopChatHandlerProvider.class) + .in(LazySingleton.class); + binder.bind(ServiceAnnouncingChatHandlerProvider.class).in(LazySingleton.class); + binder.bind(NoopChatHandlerProvider.class).in(LazySingleton.class); + + binder.bind(WorkerTaskMonitor.class).in(ManageLifecycle.class); + binder.bind(WorkerCuratorCoordinator.class).in(ManageLifecycle.class); + LifecycleModule.register(binder, WorkerTaskMonitor.class); + + // Build it to make it bind even if nothing binds to it. + Binders.dataSegmentKillerBinder(binder); + binder.bind(DataSegmentKiller.class).to(OmniDataSegmentKiller.class).in(LazySingleton.class); + Binders.dataSegmentMoverBinder(binder); + binder.bind(DataSegmentMover.class).to(OmniDataSegmentMover.class).in(LazySingleton.class); + Binders.dataSegmentArchiverBinder(binder); + binder.bind(DataSegmentArchiver.class).to(OmniDataSegmentArchiver.class).in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.realtime.cache", CacheConfig.class); + binder.install(new CacheModule()); + + JsonConfigProvider.bind( + binder, + "druid.segment.handoff", + CoordinatorBasedSegmentHandoffNotifierConfig.class + ); + binder.bind(SegmentHandoffNotifierFactory.class) + .to(CoordinatorBasedSegmentHandoffNotifierFactory.class) + .in(LazySingleton.class); + + binder.bind(AppenderatorsManager.class) + .to(UnifiedIndexerAppenderatorsManager.class) + .in(LazySingleton.class); + + binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.INDEXER_EXECUTOR)); + + binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); + Jerseys.addResource(binder, SegmentListerResource.class); + Jerseys.addResource(binder, WorkerResource.class); + Jerseys.addResource(binder, TaskManagementResource.class); + + LifecycleModule.register(binder, Server.class); + LifecycleModule.register(binder, UnifiedIndexerLifecycleHandler.class); + + bindAnnouncer( + binder, + DiscoverySideEffectsProvider.builder(NodeType.INDEXER) + .serviceClasses( + ImmutableList.of(LookupNodeService.class, WorkerNodeService.class) + ) + .build() + ); + } + + private void configureTaskActionClient(Binder binder) + { + PolyBind.createChoice( + binder, + "druid.peon.mode", + Key.get(TaskActionClientFactory.class), + Key.get(RemoteTaskActionClientFactory.class) + ); + final MapBinder taskActionBinder = + PolyBind.optionBinder(binder, Key.get(TaskActionClientFactory.class)); + taskActionBinder + .addBinding("local") + .to(LocalTaskActionClientFactory.class) + .in(LazySingleton.class); + // all of these bindings are so that we can run the peon in local mode + JsonConfigProvider.bind(binder, "druid.indexer.storage", TaskStorageConfig.class); + binder.bind(TaskStorage.class).to(HeapMemoryTaskStorage.class).in(LazySingleton.class); + binder.bind(TaskActionToolbox.class).in(LazySingleton.class); + binder.bind(IndexerMetadataStorageCoordinator.class) + .to(IndexerSQLMetadataStorageCoordinator.class) + .in(LazySingleton.class); + taskActionBinder + .addBinding("remote") + .to(RemoteTaskActionClientFactory.class) + .in(LazySingleton.class); + } + + + @Provides + @LazySingleton + public Worker getWorker(@Self DruidNode node, WorkerConfig config) + { + return new Worker( + node.getServiceScheme(), + node.getHostAndPortToUse(), + config.getIp(), + config.getCapacity(), + config.getVersion() + ); + } + + @Provides + @LazySingleton + public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) + { + return new WorkerNodeService( + workerConfig.getIp(), + workerConfig.getCapacity(), + workerConfig.getVersion() + ); + } + }, + new IndexingServiceFirehoseModule(), + new IndexingServiceTaskLogsModule(), + new QueryablePeonModule(), + new ChatHandlerServerModule(properties, true), + new LookupModule() + ); + } +} diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index eeb8ae1f0fa2..1db7e65b2162 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -57,6 +57,8 @@ import org.apache.druid.indexing.worker.http.WorkerResource; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.lookup.LookupSerdeModule; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -130,6 +132,10 @@ public void configure(Binder binder) Jerseys.addResource(binder, WorkerResource.class); Jerseys.addResource(binder, TaskManagementResource.class); + binder.bind(AppenderatorsManager.class) + .to(DummyForInjectionAppenderatorsManager.class) + .in(LazySingleton.class); + LifecycleModule.register(binder, Server.class); bindAnnouncer( diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 741b2e78ed92..1b909682bc2a 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -95,6 +95,8 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.lookup.LookupSerdeModule; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.audit.AuditManagerProvider; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; @@ -240,6 +242,11 @@ public void configure(Binder binder) Jerseys.addResource(binder, SupervisorResource.class); Jerseys.addResource(binder, HttpRemoteTaskRunnerResource.class); + + binder.bind(AppenderatorsManager.class) + .to(DummyForInjectionAppenderatorsManager.class) + .in(LazySingleton.class); + if (standalone) { LifecycleModule.register(binder, Server.class); } diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index bbf11bd9f610..e3923c66facd 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -58,6 +58,7 @@ import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; @@ -91,6 +92,8 @@ import org.apache.druid.segment.loading.OmniDataSegmentArchiver; import org.apache.druid.segment.loading.OmniDataSegmentKiller; import org.apache.druid.segment.loading.OmniDataSegmentMover; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.PeonAppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; @@ -232,7 +235,7 @@ public void configure(Binder binder) ); binder.bind(TaskReportFileWriter.class).toInstance( - new TaskReportFileWriter( + new SingleFileTaskReportFileWriter( new File(taskReportPath) ) ); @@ -255,6 +258,10 @@ public void configure(Binder binder) binder.bind(CoordinatorClient.class).in(LazySingleton.class); + binder.bind(AppenderatorsManager.class) + .to(PeonAppenderatorsManager.class) + .in(LazySingleton.class); + binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); Jerseys.addResource(binder, SegmentListerResource.class); binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(ServerType.fromString(nodeType))); @@ -333,7 +340,7 @@ public SegmentListerResource getSegmentListerResource( }, new QueryablePeonModule(), new IndexingServiceFirehoseModule(), - new ChatHandlerServerModule(properties), + new ChatHandlerServerModule(properties, false), new LookupModule() ); } diff --git a/services/src/main/java/org/apache/druid/cli/Main.java b/services/src/main/java/org/apache/druid/cli/Main.java index 0cc44274a7bb..f966a691142a 100644 --- a/services/src/main/java/org/apache/druid/cli/Main.java +++ b/services/src/main/java/org/apache/druid/cli/Main.java @@ -60,6 +60,7 @@ public static void main(String[] args) CliHistorical.class, CliBroker.class, CliOverlord.class, + CliIndexer.class, CliMiddleManager.class, CliRouter.class ); From 54c242442de0c1ec4231b6090ed341000485cbc9 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 18 Jul 2019 18:18:20 -0700 Subject: [PATCH 02/24] Fix HttpRemoteTaskRunnerTest --- .../overlord/hrtr/HttpRemoteTaskRunnerTest.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index 5eddb2e9e5bb..eb4a76e2e3e0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -89,6 +89,8 @@ public void testFreshStart() throws Exception DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( @@ -180,6 +182,8 @@ public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); Task task1 = NoopTask.create("task-id-1", 0); @@ -270,6 +274,8 @@ public void testTaskRunnerRestart() throws Exception DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); ConcurrentMap workerHolders = new ConcurrentHashMap<>(); @@ -423,6 +429,8 @@ public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); ConcurrentMap workerHolders = new ConcurrentHashMap<>(); @@ -599,6 +607,8 @@ public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); ConcurrentMap workerHolders = new ConcurrentHashMap<>(); @@ -773,6 +783,8 @@ public void testMarkWorkersLazy() throws Exception DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); Task task1 = NoopTask.create("task-id-1", 0); @@ -1208,6 +1220,8 @@ private HttpRemoteTaskRunner createTaskRunnerForTestTaskAddedOrUpdated( DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); HttpRemoteTaskRunner taskRunner = new HttpRemoteTaskRunner( From cd9f9d4a9d38b4502ec8808ac29a30defa4959e5 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 19 Jul 2019 13:28:56 -0700 Subject: [PATCH 03/24] Remove batch sanity check on PeonAppenderatorsManager --- .../realtime/appenderator/PeonAppenderatorsManager.java | 3 +-- .../initialization/jetty/ChatHandlerServerModule.java | 6 +++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 25324c80c8f9..6995efb891b7 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -100,10 +100,9 @@ public Appenderator createOfflineAppenderatorForTask( IndexMerger indexMerger ) { + // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators if (realtimeAppenderator != null) { throw new ISE("A realtime appenderator was already created for this peon's task."); - } else if (batchAppenderator != null) { - throw new ISE("A batch appenderator was already created for this peon's task."); } else { batchAppenderator = Appenderators.createOffline( schema, diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java index bb8ea39afc44..92ca51f9923a 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java @@ -125,11 +125,11 @@ public Server getServer( */ private DruidNode makeDruidNodeForSeparateChatHandler() { - String serviceName = properties.getProperty("druid.service"); - String host = properties.getProperty("druid.host"); + String serviceName = properties.getProperty("druid.service", "druid/indexer"); + String host = properties.getProperty("druid.host", DruidNode.getDefaultHost()); String bindOnHost = properties.getProperty("druid.bindOnHost", "false"); String plaintextPort = properties.getProperty("druid.plaintextPort"); - String port = properties.getProperty("druid.port"); + String port = properties.getProperty("druid.port", "8091"); String tlsPort = properties.getProperty("druid.tlsPort"); String enablePlaintextPort = properties.getProperty("druid.enablePlaintextPort", "true"); String enableTlsPort = properties.getProperty("druid.enableTlsPort", "false"); From 780a600ed7157eb76d0470f9d5f2a48203b39845 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 19 Jul 2019 14:47:54 -0700 Subject: [PATCH 04/24] Fix paralle index tests --- .../task/batch/parallel/ParallelIndexSupervisorTask.java | 8 ++++++-- .../parallel/AbstractParallelIndexSupervisorTaskTest.java | 4 +++- .../parallel/ParallelIndexSupervisorTaskSerdeTest.java | 4 +++- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 70f46fffc227..b4ea650c4842 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -53,6 +53,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -103,6 +104,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan private final ChatHandlerProvider chatHandlerProvider; private final AuthorizerMapper authorizerMapper; private final RowIngestionMetersFactory rowIngestionMetersFactory; + private final AppenderatorsManager appenderatorsManager; private final ConcurrentHashMap partitionNumCountersPerInterval = new ConcurrentHashMap<>(); @@ -120,7 +122,8 @@ public ParallelIndexSupervisorTask( @JacksonInject @Nullable IndexingServiceClient indexingServiceClient, // null in overlords @JacksonInject @Nullable ChatHandlerProvider chatHandlerProvider, // null in overlords @JacksonInject AuthorizerMapper authorizerMapper, - @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject AppenderatorsManager appenderatorsManager ) { super( @@ -143,6 +146,7 @@ public ParallelIndexSupervisorTask( this.chatHandlerProvider = chatHandlerProvider; this.authorizerMapper = authorizerMapper; this.rowIngestionMetersFactory = rowIngestionMetersFactory; + this.appenderatorsManager = appenderatorsManager; if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() != TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS) { @@ -318,7 +322,7 @@ private TaskStatus runSequential(TaskToolbox toolbox) authorizerMapper, chatHandlerProvider, rowIngestionMetersFactory, - null + appenderatorsManager ).run(toolbox); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 8792956bc95a..1644d2b5ae49 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -46,6 +46,7 @@ import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; @@ -291,7 +292,8 @@ public Authorizer getAuthorizer(String name) return new AllowAllAuthorizer(); } }, - new DropwizardRowIngestionMetersFactory() + new DropwizardRowIngestionMetersFactory(), + new TestAppenderatorsManager() ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index dd9096310d86..656023272a8c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -158,7 +159,8 @@ private ParallelIndexSupervisorTask newTask( new NoopIndexingServiceClient(), new NoopChatHandlerProvider(), new AuthorizerMapper(Collections.emptyMap()), - new DropwizardRowIngestionMetersFactory() + new DropwizardRowIngestionMetersFactory(), + new TestAppenderatorsManager() ); } } From 7fd2e4bc24b7c55e7c7ce09942c9d81a378aaf27 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 19 Jul 2019 15:32:04 -0700 Subject: [PATCH 05/24] PR comments --- .../java/org/apache/druid/guice/Jerseys.java | 4 + .../overlord/BaseRestorableTaskRunner.java | 217 +++++++++++++++++ .../indexing/overlord/ForkingTaskRunner.java | 217 +++-------------- .../overlord/ThreadingTaskRunner.java | 220 +++--------------- .../common/task/TestAppenderatorsManager.java | 2 +- ...gmentServerAnnouncerLifecycleHandler.java} | 12 +- .../appenderator/AppenderatorsManager.java | 3 + .../server/initialization/ServerConfig.java | 42 +++- .../jetty/ChatHandlerServerModule.java | 59 +---- .../jetty/CliIndexerServerModule.java | 156 +++++++++++++ .../initialization/jetty/JettyBindings.java | 29 ++- .../jetty/JettyServerInitUtils.java | 10 +- .../jetty/ServletFilterHolder.java | 6 +- .../java/org/apache/druid/cli/CliIndexer.java | 159 ++----------- .../apache/druid/cli/CliMiddleManager.java | 16 +- .../java/org/apache/druid/cli/CliPeon.java | 218 +++++++++-------- 16 files changed, 704 insertions(+), 666 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java rename server/src/main/java/org/apache/druid/segment/realtime/{UnifiedIndexerLifecycleHandler.java => CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java} (83%) create mode 100644 server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java diff --git a/core/src/main/java/org/apache/druid/guice/Jerseys.java b/core/src/main/java/org/apache/druid/guice/Jerseys.java index 08f09bdd5bc3..78f8e39cd63a 100644 --- a/core/src/main/java/org/apache/druid/guice/Jerseys.java +++ b/core/src/main/java/org/apache/druid/guice/Jerseys.java @@ -24,14 +24,18 @@ import com.google.inject.multibindings.Multibinder; import org.apache.druid.guice.annotations.JSR311Resource; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.logger.Logger; /** */ @PublicApi public class Jerseys { + private static final Logger log = new Logger(Jerseys.class); + public static void addResource(Binder binder, Class resourceClazz) { + log.info("************ adding Jersey resource: " + resourceClazz.getName()); Multibinder.newSetBinder(binder, new TypeLiteral>(){}, JSR311Resource.class) .addBinding() .toInstance(resourceClazz); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java new file mode 100644 index 000000000000..03c36d0ed3d8 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java @@ -0,0 +1,217 @@ +/* + * 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.overlord; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.io.Files; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.indexer.RunnerTaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.Executor; + +/** + * Base class for {@link ForkingTaskRunner} and {@link ThreadingTaskRunner} which support task restoration. + */ +public abstract class BaseRestorableTaskRunner implements TaskRunner +{ + protected static final EmittingLogger LOG = new EmittingLogger(BaseRestorableTaskRunner.class); + protected static final String TASK_RESTORE_FILENAME = "restore.json"; + + protected final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); + + /** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */ + protected final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); + protected final ObjectMapper jsonMapper; + protected final TaskConfig taskConfig; + + public BaseRestorableTaskRunner( + ObjectMapper jsonMapper, + TaskConfig taskConfig + ) + { + this.jsonMapper = jsonMapper; + this.taskConfig = taskConfig; + } + + @Override + public List>> restore() + { + final File restoreFile = getRestoreFile(); + final TaskRestoreInfo taskRestoreInfo; + if (restoreFile.exists()) { + try { + taskRestoreInfo = jsonMapper.readValue(restoreFile, TaskRestoreInfo.class); + } + catch (Exception e) { + LOG.error(e, "Failed to read restorable tasks from file[%s]. Skipping restore.", restoreFile); + return ImmutableList.of(); + } + } else { + return ImmutableList.of(); + } + + final List>> retVal = new ArrayList<>(); + for (final String taskId : taskRestoreInfo.getRunningTasks()) { + try { + final File taskFile = new File(taskConfig.getTaskDir(taskId), "task.json"); + final Task task = jsonMapper.readValue(taskFile, Task.class); + + if (!task.getId().equals(taskId)) { + throw new ISE("WTF?! Task[%s] restore file had wrong id[%s].", taskId, task.getId()); + } + + if (taskConfig.isRestoreTasksOnRestart() && task.canRestore()) { + LOG.info("Restoring task[%s].", task.getId()); + retVal.add(Pair.of(task, run(task))); + } + } + catch (Exception e) { + LOG.warn(e, "Failed to restore task[%s]. Trying to restore other tasks.", taskId); + } + } + + LOG.info("Restored %,d tasks.", retVal.size()); + + return retVal; + } + + @Override + public void registerListener(TaskRunnerListener listener, Executor executor) + { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listener.getListenerId())) { + throw new ISE("Listener [%s] already registered", listener.getListenerId()); + } + } + + final Pair listenerPair = Pair.of(listener, executor); + + synchronized (tasks) { + for (TaskRunnerWorkItem item : tasks.values()) { + TaskRunnerUtils.notifyLocationChanged(ImmutableList.of(listenerPair), item.getTaskId(), item.getLocation()); + } + + listeners.add(listenerPair); + LOG.info("Registered listener [%s]", listener.getListenerId()); + } + } + + @Override + public void unregisterListener(String listenerId) + { + for (Pair pair : listeners) { + if (pair.lhs.getListenerId().equals(listenerId)) { + listeners.remove(pair); + LOG.info("Unregistered listener [%s]", listenerId); + return; + } + } + } + + @Override + public Collection getRunningTasks() + { + throw new UOE("Concrete class should implement this, as it depends on TaskRunnerWorkItem implmentation details."); + } + + @Override + public Collection getPendingTasks() + { + throw new UOE("Concrete class should implement this, as it depends on TaskRunnerWorkItem implmentation details."); + } + + @Nullable + @Override + public RunnerTaskState getRunnerTaskState(String taskId) + { + throw new UOE("Concrete class should implement this, as it depends on TaskRunnerWorkItem implmentation details."); + } + + @Override + public Collection getKnownTasks() + { + synchronized (tasks) { + return Lists.newArrayList(tasks.values()); + } + } + + // Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that + // occur while saving. + @GuardedBy("tasks") + protected void saveRunningTasks() + { + final File restoreFile = getRestoreFile(); + final List theTasks = new ArrayList<>(); + for (TaskRunnerWorkItem forkingTaskRunnerWorkItem : tasks.values()) { + theTasks.add(forkingTaskRunnerWorkItem.getTaskId()); + } + + try { + Files.createParentDirs(restoreFile); + jsonMapper.writeValue(restoreFile, new TaskRestoreInfo(theTasks)); + } + catch (Exception e) { + LOG.warn(e, "Failed to save tasks to restore file[%s]. Skipping this save.", restoreFile); + } + } + + protected File getRestoreFile() + { + return new File(taskConfig.getBaseTaskDir(), TASK_RESTORE_FILENAME); + } + + protected static class TaskRestoreInfo + { + @JsonProperty + private final List runningTasks; + + @JsonCreator + public TaskRestoreInfo( + @JsonProperty("runningTasks") List runningTasks + ) + { + this.runningTasks = runningTasks; + } + + public List getRunningTasks() + { + return runningTasks; + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java index 53db87d93e01..0525f3e014c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java @@ -19,8 +19,6 @@ package org.apache.druid.indexing.overlord; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.CharMatcher; import com.google.common.base.Joiner; @@ -30,7 +28,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.io.ByteSink; import com.google.common.io.ByteSource; import com.google.common.io.ByteStreams; @@ -39,7 +36,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.commons.io.FileUtils; import org.apache.druid.guice.annotations.Self; @@ -55,7 +51,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; @@ -83,31 +78,23 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; /** * Runs tasks in separate processes using the "internal peon" verb. */ -public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer +public class ForkingTaskRunner + extends BaseRestorableTaskRunner + implements TaskLogStreamer { - private static final EmittingLogger log = new EmittingLogger(ForkingTaskRunner.class); + private static final EmittingLogger LOGGER = new EmittingLogger(ForkingTaskRunner.class); private static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property."; - private static final String TASK_RESTORE_FILENAME = "restore.json"; private final ForkingTaskRunnerConfig config; - private final TaskConfig taskConfig; private final Properties props; private final TaskLogPusher taskLogPusher; private final DruidNode node; private final ListeningExecutorService exec; - private final ObjectMapper jsonMapper; private final PortFinder portFinder; - private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); - - /** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */ - private final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); private volatile boolean stopping = false; @@ -122,11 +109,10 @@ public ForkingTaskRunner( @Self DruidNode node ) { + super(jsonMapper, taskConfig); this.config = config; - this.taskConfig = taskConfig; this.props = props; this.taskLogPusher = taskLogPusher; - this.jsonMapper = jsonMapper; this.node = node; this.portFinder = new PortFinder(config.getStartPort(), config.getEndPort(), config.getPorts()); this.exec = MoreExecutors.listeningDecorator( @@ -134,81 +120,6 @@ public ForkingTaskRunner( ); } - @Override - public List>> restore() - { - final File restoreFile = getRestoreFile(); - final TaskRestoreInfo taskRestoreInfo; - if (restoreFile.exists()) { - try { - taskRestoreInfo = jsonMapper.readValue(restoreFile, TaskRestoreInfo.class); - } - catch (Exception e) { - log.error(e, "Failed to read restorable tasks from file[%s]. Skipping restore.", restoreFile); - return ImmutableList.of(); - } - } else { - return ImmutableList.of(); - } - - final List>> retVal = new ArrayList<>(); - for (final String taskId : taskRestoreInfo.getRunningTasks()) { - try { - final File taskFile = new File(taskConfig.getTaskDir(taskId), "task.json"); - final Task task = jsonMapper.readValue(taskFile, Task.class); - - if (!task.getId().equals(taskId)) { - throw new ISE("WTF?! Task[%s] restore file had wrong id[%s].", taskId, task.getId()); - } - - if (taskConfig.isRestoreTasksOnRestart() && task.canRestore()) { - log.info("Restoring task[%s].", task.getId()); - retVal.add(Pair.of(task, run(task))); - } - } - catch (Exception e) { - log.warn(e, "Failed to restore task[%s]. Trying to restore other tasks.", taskId); - } - } - - log.info("Restored %,d tasks.", retVal.size()); - - return retVal; - } - - @Override - public void registerListener(TaskRunnerListener listener, Executor executor) - { - for (Pair pair : listeners) { - if (pair.lhs.getListenerId().equals(listener.getListenerId())) { - throw new ISE("Listener [%s] already registered", listener.getListenerId()); - } - } - - final Pair listenerPair = Pair.of(listener, executor); - - synchronized (tasks) { - for (ForkingTaskRunnerWorkItem item : tasks.values()) { - TaskRunnerUtils.notifyLocationChanged(ImmutableList.of(listenerPair), item.getTaskId(), item.getLocation()); - } - - listeners.add(listenerPair); - log.info("Registered listener [%s]", listener.getListenerId()); - } - } - - @Override - public void unregisterListener(String listenerId) - { - for (Pair pair : listeners) { - if (pair.lhs.getListenerId().equals(listenerId)) { - listeners.remove(pair); - log.info("Unregistered listener [%s]", listenerId); - return; - } - } - } - @Override public ListenableFuture run(final Task task) { @@ -257,19 +168,19 @@ public TaskStatus call() synchronized (tasks) { final ForkingTaskRunnerWorkItem taskWorkItem = tasks.get(task.getId()); - if (taskWorkItem.shutdown) { - throw new IllegalStateException("Task has been shut down!"); - } - if (taskWorkItem == null) { - log.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); + LOGGER.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); } + if (taskWorkItem.shutdown) { + throw new IllegalStateException("Task has been shut down!"); + } + if (taskWorkItem.processHolder != null) { - log.makeAlert("WTF?! TaskInfo already has a processHolder") - .addData("task", task.getId()) - .emit(); + LOGGER.makeAlert("WTF?! TaskInfo already has a processHolder") + .addData("task", task.getId()) + .emit(); throw new ISE("TaskInfo already has processHolder for task[%s]!", task.getId()); } @@ -403,7 +314,7 @@ public TaskStatus call() jsonMapper.writeValue(taskFile, task); } - log.info("Running command: %s", Joiner.on(" ").join(command)); + LOGGER.info("Running command: %s", Joiner.on(" ").join(command)); taskWorkItem.processHolder = new ProcessHolder( new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(), logFile, @@ -423,7 +334,7 @@ public TaskStatus call() TaskStatus.running(task.getId()) ); - log.info("Logging task %s output to: %s", task.getId(), logFile); + LOGGER.info("Logging task %s output to: %s", task.getId(), logFile); boolean runFailed = true; final ByteSink logSink = Files.asByteSink(logFile, FileWriteMode.APPEND); @@ -435,7 +346,7 @@ public TaskStatus call() try (final OutputStream toLogfile = logSink.openStream()) { ByteStreams.copy(processHolder.process.getInputStream(), toLogfile); final int statusCode = processHolder.process.waitFor(); - log.info("Process exited with status[%d] for task: %s", statusCode, task.getId()); + LOGGER.info("Process exited with status[%d] for task: %s", statusCode, task.getId()); if (statusCode == 0) { runFailed = false; } @@ -469,7 +380,7 @@ public TaskStatus call() } } catch (Throwable t) { - log.info(t, "Exception caught during execution"); + LOGGER.info(t, "Exception caught during execution"); throw new RuntimeException(t); } finally { @@ -493,19 +404,19 @@ public TaskStatus call() try { if (!stopping && taskDir.exists()) { - log.info("Removing task directory: %s", taskDir); + LOGGER.info("Removing task directory: %s", taskDir); FileUtils.deleteDirectory(taskDir); } } catch (Exception e) { - log.makeAlert(e, "Failed to delete task directory") - .addData("taskDir", taskDir.toString()) - .addData("task", task.getId()) - .emit(); + LOGGER.makeAlert(e, "Failed to delete task directory") + .addData("taskDir", taskDir.toString()) + .addData("task", task.getId()) + .emit(); } } catch (Exception e) { - log.error(e, "Suppressing exception caught while cleaning up task"); + LOGGER.error(e, "Suppressing exception caught while cleaning up task"); } } } @@ -528,12 +439,12 @@ public void stop() synchronized (tasks) { for (ForkingTaskRunnerWorkItem taskWorkItem : tasks.values()) { if (taskWorkItem.processHolder != null) { - log.info("Closing output stream to task[%s].", taskWorkItem.getTask().getId()); + LOGGER.info("Closing output stream to task[%s].", taskWorkItem.getTask().getId()); try { taskWorkItem.processHolder.process.getOutputStream().close(); } catch (Exception e) { - log.warn(e, "Failed to close stdout to task[%s]. Destroying task.", taskWorkItem.getTask().getId()); + LOGGER.warn(e, "Failed to close stdout to task[%s]. Destroying task.", taskWorkItem.getTask().getId()); taskWorkItem.processHolder.process.destroy(); } } @@ -544,25 +455,25 @@ public void stop() final long timeout = new Interval(start, taskConfig.getGracefulShutdownTimeout()).toDurationMillis(); // Things should be terminating now. Wait for it to happen so logs can be uploaded and all that good stuff. - log.info("Waiting up to %,dms for shutdown.", timeout); + LOGGER.info("Waiting up to %,dms for shutdown.", timeout); if (timeout > 0) { try { final boolean terminated = exec.awaitTermination(timeout, TimeUnit.MILLISECONDS); final long elapsed = System.currentTimeMillis() - start.getMillis(); if (terminated) { - log.info("Finished stopping in %,dms.", elapsed); + LOGGER.info("Finished stopping in %,dms.", elapsed); } else { final Set stillRunning; synchronized (tasks) { stillRunning = ImmutableSet.copyOf(tasks.keySet()); } - log.makeAlert("Failed to stop forked tasks") - .addData("stillRunning", stillRunning) - .addData("elapsed", elapsed) - .emit(); + LOGGER.makeAlert("Failed to stop forked tasks") + .addData("stillRunning", stillRunning) + .addData("elapsed", elapsed) + .emit(); - log.warn( + LOGGER.warn( "Executor failed to stop after %,dms, not waiting for it! Tasks still running: [%s]", elapsed, Joiner.on("; ").join(stillRunning) @@ -570,25 +481,25 @@ public void stop() } } catch (InterruptedException e) { - log.warn(e, "Interrupted while waiting for executor to finish."); + LOGGER.warn(e, "Interrupted while waiting for executor to finish."); Thread.currentThread().interrupt(); } } else { - log.warn("Ran out of time, not waiting for executor to finish!"); + LOGGER.warn("Ran out of time, not waiting for executor to finish!"); } } @Override public void shutdown(final String taskid, String reason) { - log.info("Shutdown [%s] because: [%s]", taskid, reason); + LOGGER.info("Shutdown [%s] because: [%s]", taskid, reason); final ForkingTaskRunnerWorkItem taskInfo; synchronized (tasks) { taskInfo = tasks.get(taskid); if (taskInfo == null) { - log.info("Ignoring request to cancel unknown task: %s", taskid); + LOGGER.info("Ignoring request to cancel unknown task: %s", taskid); return; } @@ -597,7 +508,7 @@ public void shutdown(final String taskid, String reason) if (taskInfo.processHolder != null) { // Will trigger normal failure mechanisms due to process exit - log.info("Killing process for task: %s", taskid); + LOGGER.info("Killing process for task: %s", taskid); taskInfo.processHolder.process.destroy(); } } @@ -630,14 +541,6 @@ public Collection getPendingTasks() } } - @Override - public Collection getKnownTasks() - { - synchronized (tasks) { - return Lists.newArrayList(tasks.values()); - } - } - @Nullable @Override public RunnerTaskState getRunnerTaskState(String taskId) @@ -694,51 +597,7 @@ public InputStream openStream() throws IOException ); } - // Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that - // occur while saving. - @GuardedBy("tasks") - private void saveRunningTasks() - { - final File restoreFile = getRestoreFile(); - final List theTasks = new ArrayList<>(); - for (ForkingTaskRunnerWorkItem forkingTaskRunnerWorkItem : tasks.values()) { - theTasks.add(forkingTaskRunnerWorkItem.getTaskId()); - } - - try { - Files.createParentDirs(restoreFile); - jsonMapper.writeValue(restoreFile, new TaskRestoreInfo(theTasks)); - } - catch (Exception e) { - log.warn(e, "Failed to save tasks to restore file[%s]. Skipping this save.", restoreFile); - } - } - - private File getRestoreFile() - { - return new File(taskConfig.getBaseTaskDir(), TASK_RESTORE_FILENAME); - } - - private static class TaskRestoreInfo - { - @JsonProperty - private final List runningTasks; - - @JsonCreator - public TaskRestoreInfo( - @JsonProperty("runningTasks") List runningTasks - ) - { - this.runningTasks = runningTasks; - } - - public List getRunningTasks() - { - return runningTasks; - } - } - - private static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem + protected static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem { private final Task task; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index 253b3bcb20be..a3e62eb1e459 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -19,21 +19,15 @@ package org.apache.druid.indexing.overlord; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; import com.google.common.io.ByteSource; -import com.google.common.io.Files; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.commons.io.FileUtils; import org.apache.druid.guice.annotations.Self; @@ -50,7 +44,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; @@ -75,13 +68,10 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; /** - * TaskRunner implemention for the Indexer task execution service, which runs all tasks in a single process. + * TaskRunner implemention for the CliIndexer task execution service, which runs all tasks in a single process. * * Two thread pools are used: * - A task execution pool, sized to number of worker slots. This is used to execute the Task run() methods. @@ -93,17 +83,15 @@ * Note that separate task logs are not supported, all task log entries will be written to the Indexer process log * instead. */ -public class ThreadingTaskRunner implements TaskRunner, TaskLogStreamer, QuerySegmentWalker +public class ThreadingTaskRunner + extends BaseRestorableTaskRunner + implements TaskLogStreamer, QuerySegmentWalker { - private static final EmittingLogger log = new EmittingLogger(ThreadingTaskRunner.class); + private static final EmittingLogger LOGGER = new EmittingLogger(ThreadingTaskRunner.class); - private static final String TASK_RESTORE_FILENAME = "restore.json"; private final TaskToolboxFactory toolboxFactory; - private final TaskConfig taskConfig; private final TaskLogPusher taskLogPusher; private final DruidNode node; - private final ObjectMapper jsonMapper; - private final CopyOnWriteArrayList> listeners = new CopyOnWriteArrayList<>(); private final AppenderatorsManager appenderatorsManager; private final TaskReportFileWriter taskReportFileWriter; private final ListeningExecutorService taskExecutor; @@ -111,9 +99,6 @@ public class ThreadingTaskRunner implements TaskRunner, TaskLogStreamer, QuerySe private volatile boolean stopping = false; - /** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */ - private final ConcurrentHashMap tasks = new ConcurrentHashMap<>(); - @Inject public ThreadingTaskRunner( TaskToolboxFactory toolboxFactory, @@ -126,10 +111,9 @@ public ThreadingTaskRunner( @Self DruidNode node ) { + super(jsonMapper, taskConfig); this.toolboxFactory = toolboxFactory; - this.taskConfig = taskConfig; this.taskLogPusher = taskLogPusher; - this.jsonMapper = jsonMapper; this.node = node; this.appenderatorsManager = appenderatorsManager; this.taskReportFileWriter = taskReportFileWriter; @@ -148,85 +132,10 @@ public Optional streamTaskLog(String taskid, long offset) throws IOE return Optional.absent(); } - @Override - public List>> restore() - { - final File restoreFile = getRestoreFile(); - final TaskRestoreInfo taskRestoreInfo; - if (restoreFile.exists()) { - try { - taskRestoreInfo = jsonMapper.readValue(restoreFile, TaskRestoreInfo.class); - } - catch (Exception e) { - log.error(e, "Failed to read restorable tasks from file[%s]. Skipping restore.", restoreFile); - return ImmutableList.of(); - } - } else { - return ImmutableList.of(); - } - - final List>> retVal = new ArrayList<>(); - for (final String taskId : taskRestoreInfo.getRunningTasks()) { - try { - final File taskFile = new File(taskConfig.getTaskDir(taskId), "task.json"); - final Task task = jsonMapper.readValue(taskFile, Task.class); - - if (!task.getId().equals(taskId)) { - throw new ISE("WTF?! Task[%s] restore file had wrong id[%s].", taskId, task.getId()); - } - - if (taskConfig.isRestoreTasksOnRestart() && task.canRestore()) { - log.info("Restoring task[%s].", task.getId()); - retVal.add(Pair.of(task, run(task))); - } - } - catch (Exception e) { - log.warn(e, "Failed to restore task[%s]. Trying to restore other tasks.", taskId); - } - } - - log.info("Restored %,d tasks.", retVal.size()); - - return retVal; - } - @Override public void start() { - - } - - @Override - public void registerListener(TaskRunnerListener listener, Executor executor) - { - for (Pair pair : listeners) { - if (pair.lhs.getListenerId().equals(listener.getListenerId())) { - throw new ISE("Listener [%s] already registered", listener.getListenerId()); - } - } - - final Pair listenerPair = Pair.of(listener, executor); - - synchronized (tasks) { - for (ThreadingTaskRunnerWorkItem item : tasks.values()) { - TaskRunnerUtils.notifyLocationChanged(ImmutableList.of(listenerPair), item.getTaskId(), item.getLocation()); - } - - listeners.add(listenerPair); - log.info("Registered listener [%s]", listener.getListenerId()); - } - } - - @Override - public void unregisterListener(String listenerId) - { - for (Pair pair : listeners) { - if (pair.lhs.getListenerId().equals(listenerId)) { - listeners.remove(pair); - log.info("Unregistered listener [%s]", listenerId); - return; - } - } + // Nothing to start. } @Override @@ -268,14 +177,14 @@ public TaskStatus call() synchronized (tasks) { taskWorkItem = tasks.get(task.getId()); - if (taskWorkItem.shutdown) { - throw new IllegalStateException("Task has been shut down!"); - } - if (taskWorkItem == null) { - log.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); + LOGGER.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); } + + if (taskWorkItem.shutdown) { + throw new IllegalStateException("Task has been shut down!"); + } } if (!taskFile.exists()) { @@ -300,7 +209,7 @@ public TaskStatus call() return task.run(toolbox); } catch (Exception e) { - log.error(e, "Task[%s] exited with exception.", task.getId()); + LOGGER.error(e, "Task[%s] exited with exception.", task.getId()); return null; } finally { @@ -339,7 +248,7 @@ public TaskStatus call() } } catch (Throwable t) { - log.info(t, "Exception caught during execution"); + LOGGER.info(t, "Exception caught during execution"); throw new RuntimeException(t); } finally { @@ -356,19 +265,19 @@ public TaskStatus call() try { if (!stopping && taskDir.exists()) { - log.info("Removing task directory: %s", taskDir); + LOGGER.info("Removing task directory: %s", taskDir); FileUtils.deleteDirectory(taskDir); } } catch (Exception e) { - log.makeAlert(e, "Failed to delete task directory") - .addData("taskDir", taskDir.toString()) - .addData("task", task.getId()) - .emit(); + LOGGER.makeAlert(e, "Failed to delete task directory") + .addData("taskDir", taskDir.toString()) + .addData("task", task.getId()) + .emit(); } } catch (Exception e) { - log.error(e, "Suppressing exception caught while cleaning up task"); + LOGGER.error(e, "Suppressing exception caught while cleaning up task"); } } } @@ -384,19 +293,19 @@ public TaskStatus call() @Override public void shutdown(String taskid, String reason) { - log.info("Shutdown [%s] because: [%s]", taskid, reason); + LOGGER.info("Shutdown [%s] because: [%s]", taskid, reason); final ThreadingTaskRunnerWorkItem taskInfo; synchronized (tasks) { taskInfo = tasks.get(taskid); if (taskInfo == null) { - log.info("Ignoring request to cancel unknown task: %s", taskid); + LOGGER.info("Ignoring request to cancel unknown task: %s", taskid); return; } if (taskInfo.shutdown) { - log.info( + LOGGER.info( "Task [%s] is already shutting down, ignoring duplicate shutdown request with reason [%s]", taskid, reason @@ -427,7 +336,7 @@ private ListenableFuture scheduleTaskShutdown(ThreadingTaskRunnerWorkItem taskIn @Override public Void call() { - log.info("Stopping thread for task: %s", taskInfo.getTaskId()); + LOGGER.info("Stopping thread for task: %s", taskInfo.getTaskId()); taskInfo.getTask().stopGracefully(taskConfig); try { @@ -443,7 +352,7 @@ public Void call() } } catch (Exception e) { - log.info(e, "Encountered exception while waiting for task [%s] shutdown", taskInfo.getTaskId()); + LOGGER.info(e, "Encountered exception while waiting for task [%s] shutdown", taskInfo.getTaskId()); if (taskInfo.thread != null) { taskInfo.thread.interrupt(); } @@ -482,13 +391,13 @@ public void stop() shutdownFuture.get(); } catch (Exception e) { - log.error(e, "Encountered exception when stopping all tasks."); + LOGGER.error(e, "Encountered exception when stopping all tasks."); } final DateTime start = DateTimes.nowUtc(); final long gracefulShutdownMillis = taskConfig.getGracefulShutdownTimeout().toStandardDuration().getMillis(); - log.info("Waiting up to %,dms for shutdown.", gracefulShutdownMillis); + LOGGER.info("Waiting up to %,dms for shutdown.", gracefulShutdownMillis); if (gracefulShutdownMillis > 0) { try { final boolean terminated = controlThreadExecutor.awaitTermination( @@ -497,18 +406,18 @@ public void stop() ); final long elapsed = System.currentTimeMillis() - start.getMillis(); if (terminated) { - log.info("Finished stopping in %,dms.", elapsed); + LOGGER.info("Finished stopping in %,dms.", elapsed); } else { final Set stillRunning; synchronized (tasks) { stillRunning = ImmutableSet.copyOf(tasks.keySet()); } - log.makeAlert("Failed to stop task threads") - .addData("stillRunning", stillRunning) - .addData("elapsed", elapsed) - .emit(); + LOGGER.makeAlert("Failed to stop task threads") + .addData("stillRunning", stillRunning) + .addData("elapsed", elapsed) + .emit(); - log.warn( + LOGGER.warn( "Executor failed to stop after %,dms, not waiting for it! Tasks still running: [%s]", elapsed, Joiner.on("; ").join(stillRunning) @@ -516,16 +425,16 @@ public void stop() } } catch (InterruptedException e) { - log.warn(e, "Interrupted while waiting for executor to finish."); + LOGGER.warn(e, "Interrupted while waiting for executor to finish."); Thread.currentThread().interrupt(); } } else { - log.warn("Ran out of time, not waiting for executor to finish!"); + LOGGER.warn("Ran out of time, not waiting for executor to finish!"); } } @Override - public Collection getRunningTasks() + public Collection getRunningTasks() { synchronized (tasks) { final List ret = new ArrayList<>(); @@ -539,7 +448,7 @@ public Collection getRunningTasks() } @Override - public Collection getPendingTasks() + public Collection getPendingTasks() { synchronized (tasks) { final List ret = new ArrayList<>(); @@ -552,14 +461,6 @@ public Collection getPendingTasks() } } - @Override - public Collection getKnownTasks() - { - synchronized (tasks) { - return Lists.newArrayList(tasks.values()); - } - } - @Nullable @Override public RunnerTaskState getRunnerTaskState(String taskId) @@ -584,32 +485,6 @@ public Optional getScalingStats() return Optional.absent(); } - - // Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that - // occur while saving. - @GuardedBy("tasks") - private void saveRunningTasks() - { - final File restoreFile = getRestoreFile(); - final List theTasks = new ArrayList<>(); - for (ThreadingTaskRunnerWorkItem threadingTaskRunnerWorkItem : tasks.values()) { - theTasks.add(threadingTaskRunnerWorkItem.getTaskId()); - } - - try { - Files.createParentDirs(restoreFile); - jsonMapper.writeValue(restoreFile, new TaskRestoreInfo(theTasks)); - } - catch (Exception e) { - log.warn(e, "Failed to save tasks to restore file[%s]. Skipping this save.", restoreFile); - } - } - - private File getRestoreFile() - { - return new File(taskConfig.getBaseTaskDir(), TASK_RESTORE_FILENAME); - } - @Override public QueryRunner getQueryRunnerForIntervals( Query query, @@ -628,26 +503,7 @@ public QueryRunner getQueryRunnerForSegments( return appenderatorsManager.getQueryRunnerForSegments(query, specs); } - private static class TaskRestoreInfo - { - @JsonProperty - private final List runningTasks; - - @JsonCreator - public TaskRestoreInfo( - @JsonProperty("runningTasks") List runningTasks - ) - { - this.runningTasks = runningTasks; - } - - public List getRunningTasks() - { - return runningTasks; - } - } - - private static class ThreadingTaskRunnerWorkItem extends TaskRunnerWorkItem + protected static class ThreadingTaskRunnerWorkItem extends TaskRunnerWorkItem { private final Task task; private volatile Thread thread; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 6eaaad5171e0..03c5490ba480 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -115,7 +115,7 @@ public Appenderator createOfflineAppenderatorForTask( @Override public void removeAppenderatorForTask(String taskId) { - + // nothing to remove } @Override diff --git a/server/src/main/java/org/apache/druid/segment/realtime/UnifiedIndexerLifecycleHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java similarity index 83% rename from server/src/main/java/org/apache/druid/segment/realtime/UnifiedIndexerLifecycleHandler.java rename to server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java index de28c3ad1d1c..b549657563fd 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/UnifiedIndexerLifecycleHandler.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java @@ -29,10 +29,16 @@ import java.io.IOException; +/** + * Ties the {@link DataSegmentServerAnnouncer} announce/unannounce to the lifecycle start and stop. + * + * Analogous to {@link org.apache.druid.server.coordination.SegmentLoadDropHandler} on the Historicals, + * but without segment cache management. + */ @ManageLifecycle -public class UnifiedIndexerLifecycleHandler +public class CliIndexerDataSegmentServerAnnouncerLifecycleHandler { - private static final EmittingLogger log = new EmittingLogger(UnifiedIndexerLifecycleHandler.class); + private static final EmittingLogger log = new EmittingLogger(CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); private final DataSegmentServerAnnouncer dataSegmentServerAnnouncer; @@ -42,7 +48,7 @@ public class UnifiedIndexerLifecycleHandler private volatile boolean started = false; @Inject - public UnifiedIndexerLifecycleHandler( + public CliIndexerDataSegmentServerAnnouncerLifecycleHandler( DataSegmentServerAnnouncer dataSegmentServerAnnouncer ) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index bec7c1d4aa2b..c4fa1794db9a 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -51,6 +51,9 @@ * useful for the Indexer where all Tasks run in the same process. * * The methods on AppenderatorsManager can be called by multiple threads. + * + * This class provides similar functionality to the {@link org.apache.druid.server.coordination.ServerManager} and + * {@link org.apache.druid.server.SegmentManager} on the Historical processes. */ public interface AppenderatorsManager { diff --git a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java index cc86003c913e..1c754647ec1a 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java +++ b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java @@ -34,12 +34,45 @@ */ public class ServerConfig { - public static final int DEFAULT_GZIP_INFLATE_BUFFER_SIZE = 4096; + public ServerConfig( + int numThreads, + int queueSize, + boolean enableRequestLimit, + @NotNull Period maxIdleTime, + long defaultQueryTimeout, + long maxScatterGatherBytes, + long maxQueryTimeout, + int maxRequestHeaderSize, + @NotNull Period gracefulShutdownTimeout, + @NotNull Period unannouncePropagationDelay, + int inflateBufferSize, + int compressionLevel + ) + { + this.numThreads = numThreads; + this.queueSize = queueSize; + this.enableRequestLimit = enableRequestLimit; + this.maxIdleTime = maxIdleTime; + this.defaultQueryTimeout = defaultQueryTimeout; + this.maxScatterGatherBytes = maxScatterGatherBytes; + this.maxQueryTimeout = maxQueryTimeout; + this.maxRequestHeaderSize = maxRequestHeaderSize; + this.gracefulShutdownTimeout = gracefulShutdownTimeout; + this.unannouncePropagationDelay = unannouncePropagationDelay; + this.inflateBufferSize = inflateBufferSize; + this.compressionLevel = compressionLevel; + } + + public ServerConfig() + { + + } + @JsonProperty @Min(1) - private int numThreads = Math.max(10, (JvmUtils.getRuntimeInfo().getAvailableProcessors() * 17) / 16 + 2) + 30; + private int numThreads = getDefaultNumThreads(); @JsonProperty @Min(1) @@ -207,4 +240,9 @@ public String toString() ", compressionLevel=" + compressionLevel + '}'; } + + public static int getDefaultNumThreads() + { + return Math.max(10, (JvmUtils.getRuntimeInfo().getAvailableProcessors() * 17) / 16 + 2) + 30; + } } diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java index 92ca51f9923a..b1ff03331f80 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/ChatHandlerServerModule.java @@ -48,12 +48,10 @@ public class ChatHandlerServerModule implements Module { private static final String MAX_CHAT_REQUESTS_PROPERTY = "druid.indexer.server.maxChatRequests"; private final Properties properties; - private final boolean useSeparatePort; - public ChatHandlerServerModule(Properties properties, boolean useSeparatePort) + public ChatHandlerServerModule(Properties properties) { this.properties = properties; - this.useSeparatePort = useSeparatePort; } @Override @@ -64,26 +62,20 @@ public void configure(Binder binder) if (properties.containsKey(MAX_CHAT_REQUESTS_PROPERTY)) { final int maxRequests = Integer.parseInt(properties.getProperty(MAX_CHAT_REQUESTS_PROPERTY)); - JettyBindings.addQosFilter(binder, "/druid/worker/v1/chat/*", maxRequests); + JettyBindings.addQosFilter( + binder, + "/druid/worker/v1/chat/*", + maxRequests + ); } Multibinder.newSetBinder(binder, ServletFilterHolder.class).addBinding().to(TaskIdResponseHeaderFilterHolder.class); - if (useSeparatePort) { - // bind a modified DruidNode that will be used by the Jetty server installed below - binder.bind(DruidNode.class) - .annotatedWith(RemoteChatHandler.class) - .toInstance(makeDruidNodeForSeparateChatHandler()); - - // this installs a separate Jetty server for chat handling - LifecycleModule.register(binder, Server.class, RemoteChatHandler.class); - } else { - /** - * We bind {@link DruidNode} annotated with {@link RemoteChatHandler} to {@literal @}{@link Self} {@link DruidNode} - * so that same Jetty Server is used for querying as well as ingestion. - */ - binder.bind(DruidNode.class).annotatedWith(RemoteChatHandler.class).to(Key.get(DruidNode.class, Self.class)); - } + /** + * We bind {@link DruidNode} annotated with {@link RemoteChatHandler} to {@literal @}{@link Self} {@link DruidNode} + * so that same Jetty Server is used for querying as well as ingestion. + */ + binder.bind(DruidNode.class).annotatedWith(RemoteChatHandler.class).to(Key.get(DruidNode.class, Self.class)); binder.bind(ServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(ServerConfig.class)); binder.bind(TLSServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(TLSServerConfig.class)); } @@ -118,33 +110,4 @@ public Server getServer( injector.getInstance(TLSCertificateChecker.class) ); } - - - /** - * @return Creates a DruidNode identical to the @Self DruidNode but with the port numbers incremented by 1. - */ - private DruidNode makeDruidNodeForSeparateChatHandler() - { - String serviceName = properties.getProperty("druid.service", "druid/indexer"); - String host = properties.getProperty("druid.host", DruidNode.getDefaultHost()); - String bindOnHost = properties.getProperty("druid.bindOnHost", "false"); - String plaintextPort = properties.getProperty("druid.plaintextPort"); - String port = properties.getProperty("druid.port", "8091"); - String tlsPort = properties.getProperty("druid.tlsPort"); - String enablePlaintextPort = properties.getProperty("druid.enablePlaintextPort", "true"); - String enableTlsPort = properties.getProperty("druid.enableTlsPort", "false"); - - DruidNode chatHandlerNode = new DruidNode( - serviceName, - host, - Boolean.parseBoolean(bindOnHost), - plaintextPort == null ? null : Integer.parseInt(plaintextPort), - port == null ? null : Integer.parseInt(port) + 1, - tlsPort == null ? null : Integer.parseInt(tlsPort) + 1, - Boolean.parseBoolean(enablePlaintextPort), - Boolean.parseBoolean(enableTlsPort) - ); - - return chatHandlerNode; - } } diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java new file mode 100644 index 000000000000..f710b6defdf7 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java @@ -0,0 +1,156 @@ +/* + * 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.server.initialization.jetty; + +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.multibindings.Multibinder; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.annotations.RemoteChatHandler; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.server.initialization.TLSServerConfig; +import org.apache.druid.server.metrics.DataSourceTaskIdHolder; +import org.apache.druid.server.security.TLSCertificateChecker; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.util.ssl.SslContextFactory; + +import java.util.Properties; + +/** + */ +public class CliIndexerServerModule implements Module +{ + private static final String SERVER_HTTP_NUM_THREADS_PROPERTY = "druid.server.http.numThreads"; + private final Properties properties; + + public CliIndexerServerModule(Properties properties) + { + this.properties = properties; + } + + @Override + public void configure(Binder binder) + { + Jerseys.addResource(binder, ChatHandlerResource.class); + LifecycleModule.register(binder, ChatHandlerResource.class); + + // Use an equal number of threads for chat handler and non-chat handler requests. + int serverHttpNumThreads; + if (properties.getProperty(SERVER_HTTP_NUM_THREADS_PROPERTY) == null) { + serverHttpNumThreads = ServerConfig.getDefaultNumThreads(); + } else { + serverHttpNumThreads = Integer.parseInt(properties.getProperty(SERVER_HTTP_NUM_THREADS_PROPERTY)); + } + + JettyBindings.addQosFilter( + binder, + "/druid/worker/v1/chat/*", + serverHttpNumThreads + ); + + String[] notChatPaths = new String[]{ + "/druid/v2/*", // QueryResource + "/status/*", // StatusResource + "/druid-internal/*", // SegmentListerResource, TaskManagementResource + "/druid/worker/v1/enable", // WorkerResource + "/druid/worker/v1/disable", // WorkerResource + "/druid/worker/v1/enabled", // WorkerResource + "/druid/worker/v1/tasks", // WorkerResource + "/druid/worker/v1/task/*", // WorkerResource + "/druid/v1/lookups/*", // LookupIntrospectionResource + "/druid-ext/*" // basic-security + }; + JettyBindings.addQosFilter( + binder, + notChatPaths, + serverHttpNumThreads + ); + + // Be aware that lookups have a 2 maxRequest QoS filter as well. + + Multibinder.newSetBinder(binder, ServletFilterHolder.class).addBinding().to(TaskIdResponseHeaderFilterHolder.class); + + /** + * We bind {@link DruidNode} annotated with {@link RemoteChatHandler} to {@literal @}{@link Self} {@link DruidNode} + * so that same Jetty Server is used for querying as well as ingestion. + */ + binder.bind(DruidNode.class).annotatedWith(RemoteChatHandler.class).to(Key.get(DruidNode.class, Self.class)); + binder.bind(ServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(ServerConfig.class)); + binder.bind(TLSServerConfig.class).annotatedWith(RemoteChatHandler.class).to(Key.get(TLSServerConfig.class)); + } + + @Provides + @LazySingleton + public TaskIdResponseHeaderFilterHolder taskIdResponseHeaderFilterHolderBuilder( + final DataSourceTaskIdHolder taskIdHolder + ) + { + return new TaskIdResponseHeaderFilterHolder("/druid/worker/v1/chat/*", taskIdHolder.getTaskId()); + } + + @Provides + @LazySingleton + @RemoteChatHandler + public Server getServer( + Injector injector, + Lifecycle lifecycle, + @RemoteChatHandler DruidNode node, + @RemoteChatHandler ServerConfig config, + @RemoteChatHandler TLSServerConfig TLSServerConfig + ) + { + return JettyServerModule.makeAndInitializeServer( + injector, + lifecycle, + node, + makeAdjustedServerConfig(config), + TLSServerConfig, + injector.getExistingBinding(Key.get(SslContextFactory.class)), + injector.getInstance(TLSCertificateChecker.class) + ); + } + + public ServerConfig makeAdjustedServerConfig(ServerConfig oldConfig) + { + return new ServerConfig( + (oldConfig.getNumThreads() * 2) + 2, + oldConfig.getQueueSize(), + oldConfig.isEnableRequestLimit(), + oldConfig.getMaxIdleTime(), + oldConfig.getDefaultQueryTimeout(), + oldConfig.getMaxScatterGatherBytes(), + oldConfig.getMaxQueryTimeout(), + oldConfig.getMaxRequestHeaderSize(), + oldConfig.getGracefulShutdownTimeout(), + oldConfig.getUnannouncePropagationDelay(), + oldConfig.getInflateBufferSize(), + oldConfig.getCompressionLevel() + ); + } +} diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyBindings.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyBindings.java index 3bcbafb6e03e..9942acfca54f 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyBindings.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyBindings.java @@ -40,7 +40,7 @@ private JettyBindings() // No instantiation. } - public static void addQosFilter(Binder binder, String path, int maxRequests) + public static void addQosFilter(Binder binder, String paths, int maxRequests) { if (maxRequests <= 0) { return; @@ -48,7 +48,18 @@ public static void addQosFilter(Binder binder, String path, int maxRequests) Multibinder.newSetBinder(binder, ServletFilterHolder.class) .addBinding() - .toInstance(new QosFilterHolder(path, maxRequests)); + .toInstance(new QosFilterHolder(new String[]{paths}, maxRequests)); + } + + public static void addQosFilter(Binder binder, String[] paths, int maxRequests) + { + if (maxRequests <= 0) { + return; + } + + Multibinder.newSetBinder(binder, ServletFilterHolder.class) + .addBinding() + .toInstance(new QosFilterHolder(paths, maxRequests)); } public static void addHandler(Binder binder, Class handlerClass) @@ -60,12 +71,12 @@ public static void addHandler(Binder binder, Class handlerCla private static class QosFilterHolder implements ServletFilterHolder { - private final String path; + private final String[] paths; private final int maxRequests; - public QosFilterHolder(String path, int maxRequests) + public QosFilterHolder(String[] paths, int maxRequests) { - this.path = path; + this.paths = paths; this.maxRequests = maxRequests; } @@ -90,7 +101,13 @@ public Map getInitParameters() @Override public String getPath() { - return path; + return null; + } + + @Override + public String[] getPaths() + { + return paths; } @Override diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java index 0b0234324a89..c40ca7b1b7f4 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java @@ -27,6 +27,7 @@ import org.eclipse.jetty.server.handler.RequestLogHandler; import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlet.FilterMapping; import org.eclipse.jetty.servlet.ServletContextHandler; import javax.ws.rs.HttpMethod; @@ -70,8 +71,15 @@ public static void addExtensionFilters(ServletContextHandler handler, Injector i holder.setInitParameters(servletFilterHolder.getInitParameters()); } - handler.addFilter(holder, servletFilterHolder.getPath(), servletFilterHolder.getDispatcherType()); + FilterMapping filterMapping = new FilterMapping(); + filterMapping.setFilterName(holder.getName()); + filterMapping.setPathSpecs(servletFilterHolder.getPaths()); + filterMapping.setDispatcherTypes(servletFilterHolder.getDispatcherType()); + + handler.getServletHandler().addFilter(holder, filterMapping); } + FilterMapping[] fms = handler.getServletHandler().getFilterMappings(); + int x = 55; } public static Handler getJettyRequestLogHandler() diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java index 373c28b5f506..61548647c646 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java @@ -38,7 +38,6 @@ @ExtensionPoint public interface ServletFilterHolder { - /** * Get the Filter object that should be added to the servlet. * @@ -74,6 +73,11 @@ public interface ServletFilterHolder */ String getPath(); + default String[] getPaths() + { + return new String[]{getPath()}; + } + /** * The dispatcher type that this Filter should apply to * diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 20207264424c..489f429b467e 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -22,21 +22,13 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Inject; -import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; -import com.google.inject.TypeLiteral; -import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Names; import io.airlift.airline.Command; -import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.client.indexing.HttpIndexingServiceClient; -import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeType; import org.apache.druid.discovery.WorkerNodeService; -import org.apache.druid.guice.Binders; -import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceModuleHelper; @@ -45,62 +37,26 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; -import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.NodeTypeConfig; -import org.apache.druid.guice.PolyBind; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; import org.apache.druid.guice.annotations.Self; -import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.TaskReportFileWriter; -import org.apache.druid.indexing.common.TaskToolboxFactory; -import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; -import org.apache.druid.indexing.common.actions.RemoteTaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -import org.apache.druid.indexing.common.actions.TaskActionToolbox; -import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; -import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; -import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClient; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClientFactory; -import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskRunner; -import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.ThreadingTaskRunner; import org.apache.druid.indexing.worker.Worker; -import org.apache.druid.indexing.worker.WorkerCuratorCoordinator; -import org.apache.druid.indexing.worker.WorkerTaskMonitor; import org.apache.druid.indexing.worker.config.WorkerConfig; -import org.apache.druid.indexing.worker.http.TaskManagementResource; -import org.apache.druid.indexing.worker.http.WorkerResource; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; -import org.apache.druid.segment.loading.DataSegmentArchiver; -import org.apache.druid.segment.loading.DataSegmentKiller; -import org.apache.druid.segment.loading.DataSegmentMover; -import org.apache.druid.segment.loading.OmniDataSegmentArchiver; -import org.apache.druid.segment.loading.OmniDataSegmentKiller; -import org.apache.druid.segment.loading.OmniDataSegmentMover; -import org.apache.druid.segment.realtime.UnifiedIndexerLifecycleHandler; +import org.apache.druid.segment.realtime.CliIndexerDataSegmentServerAnnouncerLifecycleHandler; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import org.apache.druid.segment.realtime.firehose.ServiceAnnouncingChatHandlerProvider; -import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierConfig; -import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory; -import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.http.SegmentListerResource; -import org.apache.druid.server.initialization.jetty.ChatHandlerServerModule; +import org.apache.druid.server.initialization.jetty.CliIndexerServerModule; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.eclipse.jetty.server.Server; @@ -112,7 +68,7 @@ */ @Command( name = "indexer", - description = "Runs an Indexer. Description TBD." + description = "Runs an Indexer. The Indexer is a task execution process that runs each task in a separate thread." ) public class CliIndexer extends ServerRunnable { @@ -139,15 +95,14 @@ protected List getModules() public void configure(Binder binder) { binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/indexer"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8091); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8291); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8092); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8292); IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); - JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); JsonConfigProvider.bind(binder, "druid.worker", WorkerConfig.class); - JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.class); - JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); + + CliPeon.bindTaskConfigAndClients(binder); binder.bind(TaskReportFileWriter.class).toInstance(new TaskReportFileWriter()); @@ -155,70 +110,17 @@ public void configure(Binder binder) binder.bind(QuerySegmentWalker.class).to(ThreadingTaskRunner.class); binder.bind(ThreadingTaskRunner.class).in(LazySingleton.class); - binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); - - configureTaskActionClient(binder); - binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - binder.bind(new TypeLiteral>(){}) - .to(ParallelIndexTaskClientFactory.class) - .in(LazySingleton.class); + CliPeon.bindRowIngestionMeters(binder); - PolyBind.createChoice( - binder, - "druid.indexer.task.rowIngestionMeters.type", - Key.get(RowIngestionMetersFactory.class), - Key.get(DropwizardRowIngestionMetersFactory.class) - ); - final MapBinder rowIngestionMetersHandlerProviderBinder = - PolyBind.optionBinder(binder, Key.get(RowIngestionMetersFactory.class)); - rowIngestionMetersHandlerProviderBinder - .addBinding("dropwizard") - .to(DropwizardRowIngestionMetersFactory.class) - .in(LazySingleton.class); - binder.bind(DropwizardRowIngestionMetersFactory.class).in(LazySingleton.class); + CliPeon.bindChatHandler(binder); - PolyBind.createChoice( - binder, - "druid.indexer.task.chathandler.type", - Key.get(ChatHandlerProvider.class), - Key.get(ServiceAnnouncingChatHandlerProvider.class) - ); - final MapBinder handlerProviderBinder = - PolyBind.optionBinder(binder, Key.get(ChatHandlerProvider.class)); - handlerProviderBinder - .addBinding("announce") - .to(ServiceAnnouncingChatHandlerProvider.class) - .in(LazySingleton.class); - handlerProviderBinder - .addBinding("noop") - .to(NoopChatHandlerProvider.class) - .in(LazySingleton.class); - binder.bind(ServiceAnnouncingChatHandlerProvider.class).in(LazySingleton.class); - binder.bind(NoopChatHandlerProvider.class).in(LazySingleton.class); + CliPeon.bindPeonDataSegmentHandlers(binder); - binder.bind(WorkerTaskMonitor.class).in(ManageLifecycle.class); - binder.bind(WorkerCuratorCoordinator.class).in(ManageLifecycle.class); - LifecycleModule.register(binder, WorkerTaskMonitor.class); + CliPeon.bindRealtimeCache(binder); - // Build it to make it bind even if nothing binds to it. - Binders.dataSegmentKillerBinder(binder); - binder.bind(DataSegmentKiller.class).to(OmniDataSegmentKiller.class).in(LazySingleton.class); - Binders.dataSegmentMoverBinder(binder); - binder.bind(DataSegmentMover.class).to(OmniDataSegmentMover.class).in(LazySingleton.class); - Binders.dataSegmentArchiverBinder(binder); - binder.bind(DataSegmentArchiver.class).to(OmniDataSegmentArchiver.class).in(LazySingleton.class); + CliPeon.bindCoordinatorHandoffNotiferAndClient(binder); - JsonConfigProvider.bind(binder, "druid.realtime.cache", CacheConfig.class); - binder.install(new CacheModule()); - - JsonConfigProvider.bind( - binder, - "druid.segment.handoff", - CoordinatorBasedSegmentHandoffNotifierConfig.class - ); - binder.bind(SegmentHandoffNotifierFactory.class) - .to(CoordinatorBasedSegmentHandoffNotifierFactory.class) - .in(LazySingleton.class); + CliMiddleManager.bindWorkerManagementClasses(binder); binder.bind(AppenderatorsManager.class) .to(UnifiedIndexerAppenderatorsManager.class) @@ -228,11 +130,10 @@ public void configure(Binder binder) binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); Jerseys.addResource(binder, SegmentListerResource.class); - Jerseys.addResource(binder, WorkerResource.class); - Jerseys.addResource(binder, TaskManagementResource.class); + + LifecycleModule.register(binder, CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); LifecycleModule.register(binder, Server.class); - LifecycleModule.register(binder, UnifiedIndexerLifecycleHandler.class); bindAnnouncer( binder, @@ -244,34 +145,6 @@ public void configure(Binder binder) ); } - private void configureTaskActionClient(Binder binder) - { - PolyBind.createChoice( - binder, - "druid.peon.mode", - Key.get(TaskActionClientFactory.class), - Key.get(RemoteTaskActionClientFactory.class) - ); - final MapBinder taskActionBinder = - PolyBind.optionBinder(binder, Key.get(TaskActionClientFactory.class)); - taskActionBinder - .addBinding("local") - .to(LocalTaskActionClientFactory.class) - .in(LazySingleton.class); - // all of these bindings are so that we can run the peon in local mode - JsonConfigProvider.bind(binder, "druid.indexer.storage", TaskStorageConfig.class); - binder.bind(TaskStorage.class).to(HeapMemoryTaskStorage.class).in(LazySingleton.class); - binder.bind(TaskActionToolbox.class).in(LazySingleton.class); - binder.bind(IndexerMetadataStorageCoordinator.class) - .to(IndexerSQLMetadataStorageCoordinator.class) - .in(LazySingleton.class); - taskActionBinder - .addBinding("remote") - .to(RemoteTaskActionClientFactory.class) - .in(LazySingleton.class); - } - - @Provides @LazySingleton public Worker getWorker(@Self DruidNode node, WorkerConfig config) @@ -299,7 +172,7 @@ public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) new IndexingServiceFirehoseModule(), new IndexingServiceTaskLogsModule(), new QueryablePeonModule(), - new ChatHandlerServerModule(properties, true), + new CliIndexerServerModule(properties), new LookupModule() ); } diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index 1db7e65b2162..2321e52b7894 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -121,16 +121,11 @@ public void configure(Binder binder) .in(LazySingleton.class); binder.bind(DropwizardRowIngestionMetersFactory.class).in(LazySingleton.class); + bindWorkerManagementClasses(binder); - binder.bind(WorkerTaskMonitor.class).in(ManageLifecycle.class); - binder.bind(WorkerCuratorCoordinator.class).in(ManageLifecycle.class); - - LifecycleModule.register(binder, WorkerTaskMonitor.class); binder.bind(JettyServerInitializer.class) .to(MiddleManagerJettyServerInitializer.class) .in(LazySingleton.class); - Jerseys.addResource(binder, WorkerResource.class); - Jerseys.addResource(binder, TaskManagementResource.class); binder.bind(AppenderatorsManager.class) .to(DummyForInjectionAppenderatorsManager.class) @@ -175,4 +170,13 @@ public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) new LookupSerdeModule() ); } + + public static void bindWorkerManagementClasses(Binder binder) + { + binder.bind(WorkerTaskMonitor.class).in(ManageLifecycle.class); + binder.bind(WorkerCuratorCoordinator.class).in(ManageLifecycle.class); + LifecycleModule.register(binder, WorkerTaskMonitor.class); + Jerseys.addResource(binder, WorkerResource.class); + Jerseys.addResource(binder, TaskManagementResource.class); + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index e3923c66facd..4a6c2d8ace77 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -169,62 +169,13 @@ public void configure(Binder binder) binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - PolyBind.createChoice( - binder, - "druid.indexer.task.rowIngestionMeters.type", - Key.get(RowIngestionMetersFactory.class), - Key.get(DropwizardRowIngestionMetersFactory.class) - ); - final MapBinder rowIngestionMetersHandlerProviderBinder = - PolyBind.optionBinder(binder, Key.get(RowIngestionMetersFactory.class)); - rowIngestionMetersHandlerProviderBinder - .addBinding("dropwizard") - .to(DropwizardRowIngestionMetersFactory.class) - .in(LazySingleton.class); - binder.bind(DropwizardRowIngestionMetersFactory.class).in(LazySingleton.class); - - PolyBind.createChoice( - binder, - "druid.indexer.task.chathandler.type", - Key.get(ChatHandlerProvider.class), - Key.get(ServiceAnnouncingChatHandlerProvider.class) - ); - final MapBinder handlerProviderBinder = - PolyBind.optionBinder(binder, Key.get(ChatHandlerProvider.class)); - handlerProviderBinder - .addBinding("announce") - .to(ServiceAnnouncingChatHandlerProvider.class) - .in(LazySingleton.class); - handlerProviderBinder - .addBinding("noop") - .to(NoopChatHandlerProvider.class) - .in(LazySingleton.class); - binder.bind(ServiceAnnouncingChatHandlerProvider.class).in(LazySingleton.class); - - binder.bind(NoopChatHandlerProvider.class).in(LazySingleton.class); - - binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); - - JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); - JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.class); - JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); - - configureTaskActionClient(binder); - binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - - binder.bind(new TypeLiteral>(){}) - .to(ParallelIndexTaskClientFactory.class) - .in(LazySingleton.class); + bindRowIngestionMeters(binder); - binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); + bindChatHandler(binder); - // Build it to make it bind even if nothing binds to it. - Binders.dataSegmentKillerBinder(binder); - binder.bind(DataSegmentKiller.class).to(OmniDataSegmentKiller.class).in(LazySingleton.class); - Binders.dataSegmentMoverBinder(binder); - binder.bind(DataSegmentMover.class).to(OmniDataSegmentMover.class).in(LazySingleton.class); - Binders.dataSegmentArchiverBinder(binder); - binder.bind(DataSegmentArchiver.class).to(OmniDataSegmentArchiver.class).in(LazySingleton.class); + bindTaskConfigAndClients(binder); + + bindPeonDataSegmentHandlers(binder); binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class); LifecycleModule.register(binder, ExecutorLifecycle.class); @@ -244,19 +195,9 @@ public void configure(Binder binder) binder.bind(QuerySegmentWalker.class).to(SingleTaskBackgroundRunner.class); binder.bind(SingleTaskBackgroundRunner.class).in(ManageLifecycle.class); - JsonConfigProvider.bind(binder, "druid.realtime.cache", CacheConfig.class); - binder.install(new CacheModule()); + bindRealtimeCache(binder); - JsonConfigProvider.bind( - binder, - "druid.segment.handoff", - CoordinatorBasedSegmentHandoffNotifierConfig.class - ); - binder.bind(SegmentHandoffNotifierFactory.class) - .to(CoordinatorBasedSegmentHandoffNotifierFactory.class) - .in(LazySingleton.class); - - binder.bind(CoordinatorClient.class).in(LazySingleton.class); + bindCoordinatorHandoffNotiferAndClient(binder); binder.bind(AppenderatorsManager.class) .to(PeonAppenderatorsManager.class) @@ -268,33 +209,6 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); } - private void configureTaskActionClient(Binder binder) - { - PolyBind.createChoice( - binder, - "druid.peon.mode", - Key.get(TaskActionClientFactory.class), - Key.get(RemoteTaskActionClientFactory.class) - ); - final MapBinder taskActionBinder = - PolyBind.optionBinder(binder, Key.get(TaskActionClientFactory.class)); - taskActionBinder - .addBinding("local") - .to(LocalTaskActionClientFactory.class) - .in(LazySingleton.class); - // all of these bindings are so that we can run the peon in local mode - JsonConfigProvider.bind(binder, "druid.indexer.storage", TaskStorageConfig.class); - binder.bind(TaskStorage.class).to(HeapMemoryTaskStorage.class).in(LazySingleton.class); - binder.bind(TaskActionToolbox.class).in(LazySingleton.class); - binder.bind(IndexerMetadataStorageCoordinator.class) - .to(IndexerSQLMetadataStorageCoordinator.class) - .in(LazySingleton.class); - taskActionBinder - .addBinding("remote") - .to(RemoteTaskActionClientFactory.class) - .in(LazySingleton.class); - } - @Provides @LazySingleton public Task readTask(@Json ObjectMapper mapper, ExecutorLifecycleConfig config) @@ -340,7 +254,7 @@ public SegmentListerResource getSegmentListerResource( }, new QueryablePeonModule(), new IndexingServiceFirehoseModule(), - new ChatHandlerServerModule(properties, false), + new ChatHandlerServerModule(properties), new LookupModule() ); } @@ -390,4 +304,120 @@ public void run() throw new RuntimeException(e); } } + + public static void bindRowIngestionMeters(Binder binder) + { + PolyBind.createChoice( + binder, + "druid.indexer.task.rowIngestionMeters.type", + Key.get(RowIngestionMetersFactory.class), + Key.get(DropwizardRowIngestionMetersFactory.class) + ); + final MapBinder rowIngestionMetersHandlerProviderBinder = + PolyBind.optionBinder(binder, Key.get(RowIngestionMetersFactory.class)); + rowIngestionMetersHandlerProviderBinder + .addBinding("dropwizard") + .to(DropwizardRowIngestionMetersFactory.class) + .in(LazySingleton.class); + binder.bind(DropwizardRowIngestionMetersFactory.class).in(LazySingleton.class); + } + + public static void bindChatHandler(Binder binder) + { + PolyBind.createChoice( + binder, + "druid.indexer.task.chathandler.type", + Key.get(ChatHandlerProvider.class), + Key.get(ServiceAnnouncingChatHandlerProvider.class) + ); + final MapBinder handlerProviderBinder = + PolyBind.optionBinder(binder, Key.get(ChatHandlerProvider.class)); + handlerProviderBinder + .addBinding("announce") + .to(ServiceAnnouncingChatHandlerProvider.class) + .in(LazySingleton.class); + handlerProviderBinder + .addBinding("noop") + .to(NoopChatHandlerProvider.class) + .in(LazySingleton.class); + binder.bind(ServiceAnnouncingChatHandlerProvider.class).in(LazySingleton.class); + binder.bind(NoopChatHandlerProvider.class).in(LazySingleton.class); + } + + public static void bindPeonDataSegmentHandlers(Binder binder) + { + // Build it to make it bind even if nothing binds to it. + Binders.dataSegmentKillerBinder(binder); + binder.bind(DataSegmentKiller.class).to(OmniDataSegmentKiller.class).in(LazySingleton.class); + Binders.dataSegmentMoverBinder(binder); + binder.bind(DataSegmentMover.class).to(OmniDataSegmentMover.class).in(LazySingleton.class); + Binders.dataSegmentArchiverBinder(binder); + binder.bind(DataSegmentArchiver.class).to(OmniDataSegmentArchiver.class).in(LazySingleton.class); + } + + public static void configureTaskActionClient(Binder binder) + { + PolyBind.createChoice( + binder, + "druid.peon.mode", + Key.get(TaskActionClientFactory.class), + Key.get(RemoteTaskActionClientFactory.class) + ); + final MapBinder taskActionBinder = + PolyBind.optionBinder(binder, Key.get(TaskActionClientFactory.class)); + taskActionBinder + .addBinding("local") + .to(LocalTaskActionClientFactory.class) + .in(LazySingleton.class); + // all of these bindings are so that we can run the peon in local mode + JsonConfigProvider.bind(binder, "druid.indexer.storage", TaskStorageConfig.class); + binder.bind(TaskStorage.class).to(HeapMemoryTaskStorage.class).in(LazySingleton.class); + binder.bind(TaskActionToolbox.class).in(LazySingleton.class); + binder.bind(IndexerMetadataStorageCoordinator.class) + .to(IndexerSQLMetadataStorageCoordinator.class) + .in(LazySingleton.class); + taskActionBinder + .addBinding("remote") + .to(RemoteTaskActionClientFactory.class) + .in(LazySingleton.class); + } + + public static void bindTaskConfigAndClients(Binder binder) + { + binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); + JsonConfigProvider.bind(binder, "druid.indexer.auditlog", TaskAuditLogConfig.class); + JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); + + configureTaskActionClient(binder); + binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); + + binder.bind(new TypeLiteral>(){}) + .to(ParallelIndexTaskClientFactory.class) + .in(LazySingleton.class); + + binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); + } + + public static void bindRealtimeCache(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.realtime.cache", CacheConfig.class); + binder.install(new CacheModule()); + } + + public static void bindCoordinatorHandoffNotiferAndClient(Binder binder) + { + JsonConfigProvider.bind( + binder, + "druid.segment.handoff", + CoordinatorBasedSegmentHandoffNotifierConfig.class + ); + binder.bind(SegmentHandoffNotifierFactory.class) + .to(CoordinatorBasedSegmentHandoffNotifierFactory.class) + .in(LazySingleton.class); + + binder.bind(CoordinatorClient.class).in(LazySingleton.class); + } + } From c5f7a7d94d4b780100ea4f801c540fe4dcd05ce8 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 22 Jul 2019 16:24:02 -0700 Subject: [PATCH 06/24] Adjust Jersey resource logging --- core/src/main/java/org/apache/druid/guice/Jerseys.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/guice/Jerseys.java b/core/src/main/java/org/apache/druid/guice/Jerseys.java index 78f8e39cd63a..5a9ab5b0f0c6 100644 --- a/core/src/main/java/org/apache/druid/guice/Jerseys.java +++ b/core/src/main/java/org/apache/druid/guice/Jerseys.java @@ -31,11 +31,11 @@ @PublicApi public class Jerseys { - private static final Logger log = new Logger(Jerseys.class); + private static final Logger LOG = new Logger(Jerseys.class); public static void addResource(Binder binder, Class resourceClazz) { - log.info("************ adding Jersey resource: " + resourceClazz.getName()); + LOG.info("Adding Jersey resource: " + resourceClazz.getName()); Multibinder.newSetBinder(binder, new TypeLiteral>(){}, JSR311Resource.class) .addBinding() .toInstance(resourceClazz); From 05af951927154d8af0926161a5beae1f48160c14 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 22 Jul 2019 16:34:27 -0700 Subject: [PATCH 07/24] Additional cleanup --- .../java/org/apache/druid/query/lookup/LookupModule.java | 3 ++- .../initialization/jetty/CliIndexerServerModule.java | 9 ++++++++- .../initialization/jetty/JettyServerInitUtils.java | 9 ++++++--- .../server/initialization/jetty/ServletFilterHolder.java | 8 ++++++++ 4 files changed, 24 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/query/lookup/LookupModule.java b/server/src/main/java/org/apache/druid/query/lookup/LookupModule.java index c173dc1b7528..1459663d93fe 100644 --- a/server/src/main/java/org/apache/druid/query/lookup/LookupModule.java +++ b/server/src/main/java/org/apache/druid/query/lookup/LookupModule.java @@ -46,6 +46,7 @@ public class LookupModule implements DruidModule { static final String PROPERTY_BASE = "druid.lookup"; public static final String FAILED_UPDATES_KEY = "failedUpdates"; + public static final int LOOKUP_LISTENER_QOS_MAX_REQUESTS = 2; public static String getTierListenerPath(String tier) { @@ -80,7 +81,7 @@ public void configure(Binder binder) JettyBindings.addQosFilter( binder, ListenerResource.BASE_PATH + "/" + LookupCoordinatorManager.LOOKUP_LISTEN_ANNOUNCE_KEY, - 2 // 1 for "normal" operation and 1 for "emergency" or other + LOOKUP_LISTENER_QOS_MAX_REQUESTS // 1 for "normal" operation and 1 for "emergency" or other ); } diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java index f710b6defdf7..a3a456bbd0f7 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java @@ -31,6 +31,7 @@ import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.query.lookup.LookupModule; import org.apache.druid.segment.realtime.firehose.ChatHandlerResource; import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ServerConfig; @@ -136,10 +137,16 @@ public Server getServer( ); } + /** + * Adjusts the ServerConfig such that we double the number of configured HTTP threads, + * with one half allocated using QoS to chat handler requests, and the other half for other requests. + * + * 2 dedicated threads are added for lookup listening, which also has a QoS filter applied. + */ public ServerConfig makeAdjustedServerConfig(ServerConfig oldConfig) { return new ServerConfig( - (oldConfig.getNumThreads() * 2) + 2, + (oldConfig.getNumThreads() * 2) + LookupModule.LOOKUP_LISTENER_QOS_MAX_REQUESTS, oldConfig.getQueueSize(), oldConfig.isEnableRequestLimit(), oldConfig.getMaxIdleTime(), diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java index c40ca7b1b7f4..ee1b5a6c5d1b 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/JettyServerInitUtils.java @@ -31,6 +31,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler; import javax.ws.rs.HttpMethod; +import java.util.Arrays; import java.util.Set; public class JettyServerInitUtils @@ -64,7 +65,11 @@ public static void addExtensionFilters(ServletContextHandler handler, Injector i } else if (servletFilterHolder.getFilterClass() != null) { holder = new FilterHolder(servletFilterHolder.getFilterClass()); } else { - throw new ISE("Filter[%s] for path[%s] didn't have a Filter!?", servletFilterHolder, servletFilterHolder.getPath()); + throw new ISE( + "Filter[%s] for paths[%s] didn't have a Filter!?", + servletFilterHolder, + Arrays.toString(servletFilterHolder.getPaths()) + ); } if (servletFilterHolder.getInitParameters() != null) { @@ -78,8 +83,6 @@ public static void addExtensionFilters(ServletContextHandler handler, Injector i handler.getServletHandler().addFilter(holder, filterMapping); } - FilterMapping[] fms = handler.getServletHandler().getFilterMappings(); - int x = 55; } public static Handler getJettyRequestLogHandler() diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java index 61548647c646..a3a43a18a1f7 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/ServletFilterHolder.java @@ -67,12 +67,20 @@ public interface ServletFilterHolder Map getInitParameters(); /** + * This method is deprecated, please implement {@link #getPaths()}. + * * The path that this Filter should apply to * * @return the path that this Filter should apply to */ + @Deprecated String getPath(); + /** + * The paths that this Filter should apply to + * + * @return the paths that this Filter should apply to + */ default String[] getPaths() { return new String[]{getPath()}; From f4cbf148f514dd393117fc670321212af57afc8d Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 22 Jul 2019 17:12:16 -0700 Subject: [PATCH 08/24] Fix SystemSchemaTest --- .../sql/calcite/schema/SystemSchemaTest.java | 43 +++++++++++++++---- 1 file changed, 34 insertions(+), 9 deletions(-) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index ff98ef2517c3..34007b1fae78 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -429,6 +429,13 @@ DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerT DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)) ); + private final DiscoveryDruidNode indexer = new DiscoveryDruidNode( + new DruidNode("s8", "indexerHost", false, 8092, null, true, false), + NodeType.INDEXER, + ImmutableMap.of( + DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0)) + ); + private final ImmutableDruidServer druidServer1 = new ImmutableDruidServer( new DruidServerMetadata("server1", "localhost:0000", null, 5L, ServerType.REALTIME, DruidServer.DEFAULT_TIER, 0), 1L, @@ -694,6 +701,8 @@ public void testServersTable() final DruidNodeDiscovery historicalNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); final DruidNodeDiscovery mmNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); final DruidNodeDiscovery peonNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); + final DruidNodeDiscovery indexerNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.COORDINATOR)) .andReturn(coordinatorNodeDiscovery) @@ -709,6 +718,9 @@ public void testServersTable() EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) .andReturn(mmNodeDiscovery) .once(); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + .andReturn(indexerNodeDiscovery) + .once(); EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.PEON)).andReturn(peonNodeDiscovery).once(); EasyMock.expect(coordinatorNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(coordinator)).once(); @@ -718,6 +730,7 @@ public void testServersTable() EasyMock.expect(historicalNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(historical1, historical2)).once(); EasyMock.expect(mmNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(middleManager)).once(); EasyMock.expect(peonNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(peon1, peon2)).once(); + EasyMock.expect(indexerNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(indexer)).once(); final DruidServer server1 = EasyMock.createMock(DruidServer.class); EasyMock.expect(serverInventoryView.getInventoryValue(historical1.toDruidServer().getName())).andReturn(server1).once(); @@ -734,7 +747,8 @@ public void testServersTable() routerNodeDiscovery, historicalNodeDiscovery, mmNodeDiscovery, - peonNodeDiscovery + peonNodeDiscovery, + indexerNodeDiscovery ); DataContext dataContext = new DataContext() @@ -765,7 +779,7 @@ public Object get(String name) }; final List rows = serversTable.scan(dataContext).toList(); rows.sort((Object[] row1, Object[] row2) -> ((Comparable) row1[0]).compareTo(row2[0])); - Assert.assertEquals(10, rows.size()); + Assert.assertEquals(11, rows.size()); verifyServerRow( rows.get(0), "brokerHost:8082", @@ -790,6 +804,17 @@ public Object get(String name) ); verifyServerRow( rows.get(2), + "indexerHost:8092", + "indexerHost", + 8092, + -1, + "indexer", + null, + 0, + 0 + ); + verifyServerRow( + rows.get(3), "localhost:8080", "localhost", 8080, @@ -800,7 +825,7 @@ public Object get(String name) 0 ); verifyServerRow( - rows.get(3), + rows.get(4), "localhost:8081", "localhost", 8081, @@ -811,7 +836,7 @@ public Object get(String name) 0 ); verifyServerRow( - rows.get(4), + rows.get(5), "localhost:8082", "localhost", 8082, @@ -822,7 +847,7 @@ public Object get(String name) 0 ); verifyServerRow( - rows.get(5), + rows.get(6), "localhost:8083", "localhost", 8083, @@ -833,7 +858,7 @@ public Object get(String name) 1000 ); verifyServerRow( - rows.get(6), + rows.get(7), "localhost:8090", "localhost", 8090, @@ -844,7 +869,7 @@ public Object get(String name) 0 ); verifyServerRow( - rows.get(7), + rows.get(8), "localhost:8888", "localhost", 8888, @@ -855,7 +880,7 @@ public Object get(String name) 0 ); verifyServerRow( - rows.get(8), + rows.get(9), "mmHost:8091", "mmHost", 8091, @@ -866,7 +891,7 @@ public Object get(String name) 0 ); verifyServerRow( - rows.get(9), + rows.get(10), "peonHost:8080", "peonHost", 8080, From d3e18f919b142e0dd3aa1219acbacf9882718898 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 22 Jul 2019 17:35:11 -0700 Subject: [PATCH 09/24] Add comment to LocalDataSegmentPusherTest absolute path test --- .../druid/segment/loading/LocalDataSegmentPusherTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java b/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java index c49a548fe86a..c61e9637e94c 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java @@ -165,6 +165,8 @@ public void testPathForHadoopAbsolute() { config.storageDirectory = new File("/druid"); + // If this test fails because the path is returned as "file:/druid/", this can happen + // when a /druid directory exists on the local filesystem. Assert.assertEquals( "file:/druid", new LocalDataSegmentPusher(config).getPathForHadoop() From f7b182f06309aa136695604728353c6660ea25b0 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 22 Jul 2019 17:51:13 -0700 Subject: [PATCH 10/24] More PR comments --- .../indexing/common/task/TestAppenderatorsManager.java | 5 ----- ...exerDataSegmentServerAnnouncerLifecycleHandler.java | 10 +++++----- .../UnifiedIndexerAppenderatorsManager.java | 8 +------- 3 files changed, 6 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index 03c5490ba480..28b15eadfb3f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -46,11 +46,6 @@ public class TestAppenderatorsManager implements AppenderatorsManager { private Appenderator realtimeAppenderator; - public TestAppenderatorsManager() - { - - } - @Override public Appenderator createRealtimeAppenderatorForTask( String taskId, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java index b549657563fd..1f1636fc543c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java @@ -38,7 +38,7 @@ @ManageLifecycle public class CliIndexerDataSegmentServerAnnouncerLifecycleHandler { - private static final EmittingLogger log = new EmittingLogger(CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); + private static final EmittingLogger LOG = new EmittingLogger(CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); private final DataSegmentServerAnnouncer dataSegmentServerAnnouncer; @@ -63,7 +63,7 @@ public void start() throws IOException return; } - log.info("Starting..."); + LOG.info("Starting..."); try { dataSegmentServerAnnouncer.announce(); } @@ -72,7 +72,7 @@ public void start() throws IOException throw new RuntimeException(e); } started = true; - log.info("Started."); + LOG.info("Started."); } } @@ -84,7 +84,7 @@ public void stop() return; } - log.info("Stopping..."); + LOG.info("Stopping..."); try { dataSegmentServerAnnouncer.unannounce(); } @@ -94,7 +94,7 @@ public void stop() finally { started = false; } - log.info("Stopped."); + LOG.info("Stopped."); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 64c45ba8671c..a04714aa533d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -27,7 +27,6 @@ import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.guice.annotations.Processing; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; @@ -59,10 +58,7 @@ */ public class UnifiedIndexerAppenderatorsManager implements AppenderatorsManager { - private static final Logger log = new Logger(UnifiedIndexerAppenderatorsManager.class); - private final ConcurrentHashMap datasourceSegmentWalkers = new ConcurrentHashMap<>(); - private final ConcurrentHashMap taskAppenderatorMap = new ConcurrentHashMap<>(); private final ExecutorService queryExecutorService; private final Cache cache; @@ -137,7 +133,6 @@ public Appenderator createRealtimeAppenderatorForTask( cache ); - taskAppenderatorMap.put(taskId, appenderator); return appenderator; } @@ -162,14 +157,13 @@ public Appenderator createOfflineAppenderatorForTask( indexIO, indexMerger ); - taskAppenderatorMap.put(taskId, appenderator); return appenderator; } @Override public void removeAppenderatorForTask(String taskId) { - taskAppenderatorMap.remove(taskId); + // nothing to remove presently } @Override From 37ef2aaf027d853189b80248970b409be230dca6 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 22 Jul 2019 18:28:12 -0700 Subject: [PATCH 11/24] Use Server annotated with RemoteChatHandler --- services/src/main/java/org/apache/druid/cli/CliIndexer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index fbf2d042bf6e..123e70d11641 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -41,6 +41,7 @@ import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; +import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.guice.annotations.Self; import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.overlord.TaskRunner; @@ -136,7 +137,7 @@ public void configure(Binder binder) Jerseys.addResource(binder, ShuffleResource.class); - LifecycleModule.register(binder, Server.class); + LifecycleModule.register(binder, Server.class, RemoteChatHandler.class); bindAnnouncer( binder, From a748e26f7a2ae9cbb8946604009fb2afdadbb602 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Tue, 23 Jul 2019 15:10:48 -0700 Subject: [PATCH 12/24] More PR comments --- docs/content/querying/lookups.md | 2 ++ .../overlord/BaseRestorableTaskRunner.java | 27 ++++++++++--------- .../appenderator/AppenderatorsManager.java | 8 +++--- .../PeonAppenderatorsManager.java | 15 ++++++++++- .../server/initialization/ServerConfig.java | 5 ++++ .../java/org/apache/druid/cli/CliIndexer.java | 4 +-- .../sql/calcite/schema/SystemSchemaTest.java | 6 ++--- 7 files changed, 45 insertions(+), 22 deletions(-) diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 7af2bcd790f3..31c56f875900 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -132,6 +132,8 @@ The configuration is propagated to the query serving processes (Broker / Router The query serving processes have an internal API for managing lookups on the process and those are used by the Coordinator. The Coordinator periodically checks if any of the processes need to load/drop lookups and updates them appropriately. +Please note that only 2 simultaneous lookup configuration propagation requests can be concurrently handled by a single query serving process, to prevent lookup handling from consuming too many server HTTP connections. + # API for configuring lookups ## Bulk update diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java index 03c36d0ed3d8..a681e90714d0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java @@ -144,24 +144,27 @@ public void unregisterListener(String listenerId) } } + /** + * The concrete implementation should override this, as the implemention depends on the specifics of the + * work item implementation. + */ @Override - public Collection getRunningTasks() - { - throw new UOE("Concrete class should implement this, as it depends on TaskRunnerWorkItem implmentation details."); - } + public abstract Collection getRunningTasks(); + /** + * The concrete implementation should override this, as the implemention depends on the specifics of the + * work item implementation. + */ @Override - public Collection getPendingTasks() - { - throw new UOE("Concrete class should implement this, as it depends on TaskRunnerWorkItem implmentation details."); - } + public abstract Collection getPendingTasks(); + /** + * The concrete implementation should override this, as the implemention depends on the specifics of the + * work item implementation. + */ @Nullable @Override - public RunnerTaskState getRunnerTaskState(String taskId) - { - throw new UOE("Concrete class should implement this, as it depends on TaskRunnerWorkItem implmentation details."); - } + public abstract RunnerTaskState getRunnerTaskState(String taskId); @Override public Collection getKnownTasks() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index c4fa1794db9a..06d3f6e413e6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -39,13 +39,13 @@ import java.util.concurrent.ExecutorService; /** - * This interface defines entities that create and manage potentially multiple Appenderators. + * This interface defines entities that create and manage potentially multiple {@link Appenderator} instances. * - * The AppenderatorsManager should be used by a Task running in a Peon or an Indexer process when it needs + * The AppenderatorsManager should be used by tasks running in a Peon or an CliIndexer process when it needs * an Appenderator. * - * The AppenderatorsManager also provides methods for creating QueryRunners that read the data held by the - * Appenderators created through the AppenderatorsManager. + * The AppenderatorsManager also provides methods for creating {@link QueryRunner} instances that read the data + * held by the Appenderators created through the AppenderatorsManager. * * In later updates, this interface will be used to manage memory usage across multiple Appenderators, * useful for the Indexer where all Tasks run in the same process. diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index 6995efb891b7..7563b15ef716 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -39,6 +39,19 @@ import java.util.concurrent.ExecutorService; +/** + * Manages Appenderators for tasks running within a CliPeon process. + * + * It provides the ability to create a realtime appenderator or multiple batch appenderators, + * and serves queries on the realtime appenderator. + * + * The implementation contains sanity checks that throw errors if more than one realtime appenderator is created, + * or if a task tries to create both realtime and batch appenderators. These checks can be adjusted if these + * assumptions are no longer true. + * + * Because the peon is a separate process that will terminate after task completion, this implementation + * relies on process shutdown for resource cleanup. + */ public class PeonAppenderatorsManager implements AppenderatorsManager { private Appenderator realtimeAppenderator; @@ -120,7 +133,7 @@ public Appenderator createOfflineAppenderatorForTask( @Override public void removeAppenderatorForTask(String taskId) { - // only one appenderator and task, and the process will shutdown later, don't need to do anything + // the peon only runs one task, and the process will shutdown later, don't need to do anything } @Override diff --git a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java index 1c754647ec1a..fcd2ccf7b53d 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java +++ b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java @@ -36,6 +36,11 @@ public class ServerConfig { public static final int DEFAULT_GZIP_INFLATE_BUFFER_SIZE = 4096; + /** + * The ServerConfig is normally created using {@link org.apache.druid.guice.JsonConfigProvider} binding. + * + * This constructor is provided for callers that need to create a ServerConfig object with specific field values. + */ public ServerConfig( int numThreads, int queueSize, diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 123e70d11641..3bd45cb6e59f 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -97,8 +97,8 @@ protected List getModules() public void configure(Binder binder) { binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/indexer"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8092); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8292); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8091); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8091); IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 34007b1fae78..f79b834eb6c8 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -430,7 +430,7 @@ DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerT ); private final DiscoveryDruidNode indexer = new DiscoveryDruidNode( - new DruidNode("s8", "indexerHost", false, 8092, null, true, false), + new DruidNode("s8", "indexerHost", false, 8091, null, true, false), NodeType.INDEXER, ImmutableMap.of( DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0)) @@ -804,9 +804,9 @@ public Object get(String name) ); verifyServerRow( rows.get(2), - "indexerHost:8092", + "indexerHost:8091", "indexerHost", - 8092, + 8091, -1, "indexer", null, From a3cc12f6ba9e45271bb9e765c347e0e14faf6143 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Tue, 23 Jul 2019 15:40:35 -0700 Subject: [PATCH 13/24] Checkstyle --- .../apache/druid/indexing/overlord/BaseRestorableTaskRunner.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java index a681e90714d0..0ec246c90872 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.emitter.EmittingLogger; import javax.annotation.Nullable; From 564bbce74dea70b48101076963b49be62029b652 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 24 Jul 2019 11:05:50 -0700 Subject: [PATCH 14/24] PR comments --- docs/content/querying/lookups.md | 2 +- .../indexing/overlord/BaseRestorableTaskRunner.java | 9 ++++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/content/querying/lookups.md b/docs/content/querying/lookups.md index 31c56f875900..aa20d6486a33 100644 --- a/docs/content/querying/lookups.md +++ b/docs/content/querying/lookups.md @@ -132,7 +132,7 @@ The configuration is propagated to the query serving processes (Broker / Router The query serving processes have an internal API for managing lookups on the process and those are used by the Coordinator. The Coordinator periodically checks if any of the processes need to load/drop lookups and updates them appropriately. -Please note that only 2 simultaneous lookup configuration propagation requests can be concurrently handled by a single query serving process, to prevent lookup handling from consuming too many server HTTP connections. +Please note that only 2 simultaneous lookup configuration propagation requests can be concurrently handled by a single query serving process. This limit is applied to prevent lookup handling from consuming too many server HTTP connections. # API for configuring lookups diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java index 0ec246c90872..15a83321f25a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java @@ -45,7 +45,8 @@ import java.util.concurrent.Executor; /** - * Base class for {@link ForkingTaskRunner} and {@link ThreadingTaskRunner} which support task restoration. + * Base class for {@link ForkingTaskRunner} and {@link ThreadingTaskRunner} which support task rest + * oration. */ public abstract class BaseRestorableTaskRunner implements TaskRunner { @@ -173,8 +174,10 @@ public Collection getKnownTasks() } } - // Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that - // occur while saving. + /** + * Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that + * occur while saving. + */ @GuardedBy("tasks") protected void saveRunningTasks() { From 6de3d1fffe4347a74993013d637f843b5a2afa0b Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 24 Jul 2019 14:55:44 -0700 Subject: [PATCH 15/24] Add task shutdown to stopGracefully --- .../AppenderatorDriverRealtimeIndexTask.java | 8 +++ .../indexing/common/task/HadoopIndexTask.java | 24 ++++++++- .../druid/indexing/common/task/IndexTask.java | 37 +++++++++++++ .../common/task/RealtimeIndexTask.java | 8 +++ .../batch/parallel/ParallelIndexSubTask.java | 40 ++++++++++++-- .../parallel/ParallelIndexSubTaskSpec.java | 4 +- .../parallel/ParallelIndexSupervisorTask.java | 53 +++++++++++++------ .../SeekableStreamIndexTask.java | 2 + .../SeekableStreamIndexTaskRunner.java | 6 +++ .../ParallelIndexSupervisorTaskKillTest.java | 4 +- ...rallelIndexSupervisorTaskResourceTest.java | 4 +- .../ParallelIndexSupervisorTaskTest.java | 7 ++- 12 files changed, 170 insertions(+), 27 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 9ce27c89a41a..788e39b10d59 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -435,6 +435,14 @@ public void stopGracefully(TaskConfig taskConfig) catch (Exception e) { throw new RuntimeException(e); } + } else { + synchronized (this) { + if (!gracefullyStopped) { + // If task restore is not enabled, just interrupt immediately. + gracefullyStopped = true; + runThread.interrupt(); + } + } } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 7e8b5e49a880..6433a02e67d5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -124,6 +124,13 @@ private static String getTheDataSource(HadoopIngestionSpec spec) @JsonIgnore private String errorMsg; + @JsonIgnore + private Thread runThread; + + @JsonIgnore + private boolean stopped = false; + + /** * @param spec is used by the HadoopDruidIndexerJob to set up the appropriate parameters * for creating Druid index segments. It may be modified. @@ -231,6 +238,14 @@ public String getHadoopJobIdFileName() @Override public TaskStatus run(TaskToolbox toolbox) { + synchronized (this) { + if (stopped) { + return TaskStatus.failure(getId()); + } else { + runThread = Thread.currentThread(); + } + } + try { taskConfig = toolbox.getConfig(); if (chatHandlerProvider.isPresent()) { @@ -429,6 +444,13 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception @Override public void stopGracefully(TaskConfig taskConfig) { + synchronized (this) { + stopped = true; + if (runThread == null) { + // didn't actually start, just return + return; + } + } // To avoid issue of kill command once the ingestion task is actually completed if (!ingestionState.equals(IngestionState.COMPLETED)) { final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); @@ -463,9 +485,9 @@ public void stopGracefully(TaskConfig taskConfig) } finally { Thread.currentThread().setContextClassLoader(oldLoader); + runThread.interrupt(); } } - } @GET diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index b19695e8a9d2..24bf78ce75b0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -51,6 +51,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; @@ -181,6 +182,15 @@ private static String makeGroupId(boolean isAppendToExisting, String dataSource) @JsonIgnore private final AppenderatorsManager appenderatorsManager; + @JsonIgnore + private Thread runThread; + + @JsonIgnore + private boolean stopped = false; + + @JsonIgnore + private Appenderator appenderator; + @JsonCreator public IndexTask( @JsonProperty("id") final String id, @@ -409,6 +419,14 @@ public IndexIngestionSpec getIngestionSchema() @Override public TaskStatus run(final TaskToolbox toolbox) { + synchronized (this) { + if (stopped) { + return TaskStatus.failure(getId()); + } else { + runThread = Thread.currentThread(); + } + } + try { if (chatHandlerProvider.isPresent()) { log.info("Found chat handler of class[%s]", chatHandlerProvider.get().getClass().getName()); @@ -500,6 +518,23 @@ public TaskStatus run(final TaskToolbox toolbox) } } + @Override + public void stopGracefully(TaskConfig taskConfig) + { + synchronized (this) { + stopped = true; + // Nothing else to do for native batch except terminate + if (ingestionState != IngestionState.COMPLETED) { + if (appenderator != null) { + appenderator.closeNow(); + } + if (runThread != null) { + runThread.interrupt(); + } + } + } + } + private Map getTaskCompletionReports() { return TaskReport.buildTaskReports( @@ -939,6 +974,8 @@ private TaskStatus generateAndPublishSegments( final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { + this.appenderator = appenderator; + driver.startJob(); while (firehose.hasMore()) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 2786805b9f4c..55346e057d70 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -518,6 +518,14 @@ public void stopGracefully(TaskConfig taskConfig) catch (Exception e) { throw new RuntimeException(e); } + } else { + synchronized (this) { + if (!gracefullyStopped) { + // If task restore is not enabled, just interrupt immediately. + gracefullyStopped = true; + runThread.interrupt(); + } + } } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index cf53f15f4ccf..2f7dd97db120 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTask; @@ -58,7 +59,7 @@ import org.apache.druid.segment.realtime.RealtimeMetricsMonitor; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import org.apache.druid.segment.realtime.appenderator.Appenderators; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; @@ -93,6 +94,11 @@ public class ParallelIndexSubTask extends AbstractTask private final String supervisorTaskId; private final IndexingServiceClient indexingServiceClient; private final IndexTaskClientFactory taskClientFactory; + private final AppenderatorsManager appenderatorsManager; + + private Appenderator appenderator; + private Thread runThread; + private boolean stopped = false; @JsonCreator public ParallelIndexSubTask( @@ -105,7 +111,8 @@ public ParallelIndexSubTask( @JsonProperty("spec") final ParallelIndexIngestionSpec ingestionSchema, @JsonProperty("context") final Map context, @JacksonInject IndexingServiceClient indexingServiceClient, - @JacksonInject IndexTaskClientFactory taskClientFactory + @JacksonInject IndexTaskClientFactory taskClientFactory, + @JacksonInject AppenderatorsManager appenderatorsManager ) { super( @@ -125,6 +132,7 @@ public ParallelIndexSubTask( this.supervisorTaskId = supervisorTaskId; this.indexingServiceClient = indexingServiceClient; this.taskClientFactory = taskClientFactory; + this.appenderatorsManager = appenderatorsManager; } @Override @@ -182,6 +190,14 @@ public String getSupervisorTaskId() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { + synchronized (this) { + if (stopped) { + return TaskStatus.failure(getId()); + } else { + runThread = Thread.currentThread(); + } + } + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); @@ -308,6 +324,7 @@ private List generateAndPushSegments( final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { + this.appenderator = appenderator; driver.startJob(); final List pushedSegments = new ArrayList<>(); @@ -377,14 +394,15 @@ private List generateAndPushSegments( } } - private static Appenderator newAppenderator( + private Appenderator newAppenderator( FireDepartmentMetrics metrics, TaskToolbox toolbox, DataSchema dataSchema, ParallelIndexTuningConfig tuningConfig ) { - return Appenderators.createOffline( + return appenderatorsManager.createOfflineAppenderatorForTask( + getId(), dataSchema, tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), metrics, @@ -408,4 +426,18 @@ private static BatchAppenderatorDriver newDriver( toolbox.getDataSegmentKiller() ); } + + @Override + public void stopGracefully(TaskConfig taskConfig) + { + synchronized (this) { + stopped = true; + if (appenderator != null) { + appenderator.closeNow(); + } + if (runThread != null) { + runThread.interrupt(); + } + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java index ee64f78483ad..13938651b32c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import java.util.Map; @@ -61,7 +62,8 @@ public ParallelIndexSubTask newSubTask(int numAttempts) getIngestionSpec(), getContext(), null, - null + null, + new DummyForInjectionAppenderatorsManager() ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index efdabda8db02..bbd943483611 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -109,6 +109,9 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan private final ConcurrentHashMap partitionNumCountersPerInterval = new ConcurrentHashMap<>(); private volatile ParallelIndexTaskRunner runner; + private volatile IndexTask sequentialIndexTask; + + private boolean stopped = false; // toolbox is initlized when run() is called, and can be used for processing HTTP endpoint requests. private volatile TaskToolbox toolbox; @@ -243,8 +246,13 @@ static boolean isReady(TaskActionClient actionClient, SortedSet interv @Override public void stopGracefully(TaskConfig taskConfig) { + synchronized (this) { + stopped = true; + } if (runner != null) { runner.stopGracefully(); + } else if (sequentialIndexTask != null) { + sequentialIndexTask.stopGracefully(taskConfig); } } @@ -302,28 +310,39 @@ void setToolbox(TaskToolbox toolbox) private TaskStatus runParallel(TaskToolbox toolbox) throws Exception { - createRunner(toolbox); + synchronized (this) { + if (stopped) { + return TaskStatus.failure(getId()); + } + createRunner(toolbox); + } return TaskStatus.fromCode(getId(), Preconditions.checkNotNull(runner, "runner").run()); } private TaskStatus runSequential(TaskToolbox toolbox) { - return new IndexTask( - getId(), - getGroupId(), - getTaskResource(), - getDataSource(), - new IndexIngestionSpec( - getIngestionSchema().getDataSchema(), - getIngestionSchema().getIOConfig(), - convertToIndexTuningConfig(getIngestionSchema().getTuningConfig()) - ), - getContext(), - authorizerMapper, - chatHandlerProvider, - rowIngestionMetersFactory, - appenderatorsManager - ).run(toolbox); + synchronized (this) { + if (stopped) { + return TaskStatus.failure(getId()); + } + sequentialIndexTask = new IndexTask( + getId(), + getGroupId(), + getTaskResource(), + getDataSource(), + new IndexIngestionSpec( + getIngestionSchema().getDataSchema(), + getIngestionSchema().getIOConfig(), + convertToIndexTuningConfig(getIngestionSchema().getTuningConfig()) + ), + getContext(), + authorizerMapper, + chatHandlerProvider, + rowIngestionMetersFactory, + appenderatorsManager + ); + } + return sequentialIndexTask.run(toolbox); } private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningConfig tuningConfig) 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 41b891ffb9d8..fcc949774a93 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 @@ -182,6 +182,8 @@ public void stopGracefully(TaskConfig taskConfig) { if (taskConfig.isRestoreTasksOnRestart()) { getRunner().stopGracefully(); + } else { + getRunner().stopForcefully(); } } 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 e0a9c8242a25..45670b69cd44 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 @@ -1336,6 +1336,12 @@ public RowIngestionMeters getRowIngestionMeters() return rowIngestionMeters; } + public void stopForcefully() + { + log.info("Stopping forcefully (status: [%s])", status); + stopRequested.set(true); + runThread.interrupt(); + } public void stopGracefully() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index c7c8d0b897e3..2d0678a348d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; @@ -396,7 +397,8 @@ private TestParallelIndexSubTask( ingestionSchema, context, indexingServiceClient, - taskClientFactory + taskClientFactory, + new TestAppenderatorsManager() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 04aa5a7a1b07..17c1c8152bc3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -645,7 +646,8 @@ private class TestSubTask extends ParallelIndexSubTask ingestionSchema, context, null, - taskClientFactory + taskClientFactory, + new TestAppenderatorsManager() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index cf7dd372bfdd..4eba166cd389 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -27,6 +27,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -119,7 +120,8 @@ public void testIsReady() throws Exception spec.getIngestionSpec(), spec.getContext(), indexingServiceClient, - null + null, + new TestAppenderatorsManager() ); final TaskActionClient subTaskActionClient = createActionClient(subTask); prepareTaskForLocking(subTask); @@ -457,7 +459,8 @@ public ParallelIndexSubTask newSubTask(int numAttempts) getIngestionSpec(), getContext(), null, - new LocalParallelIndexTaskClientFactory(supervisorTask) + new LocalParallelIndexTaskClientFactory(supervisorTask), + new TestAppenderatorsManager() ); } } From c68bee6470d189afbec4035d59bad9b4648319ba Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 24 Jul 2019 20:04:37 -0700 Subject: [PATCH 16/24] Small cleanup --- .../apache/druid/indexing/common/task/IndexTaskTest.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index b3d27326f6a1..d3150aeb5249 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -141,8 +141,6 @@ public static Iterable constructorFeeder() private static final IndexSpec indexSpec = new IndexSpec(); private final ObjectMapper jsonMapper; - private volatile int segmentAllocatePartitionCounter; - private File reportsFile; private AppenderatorsManager appenderatorsManager; private final IndexIO indexIO; private final RowIngestionMetersFactory rowIngestionMetersFactory; @@ -162,11 +160,7 @@ public IndexTaskTest(LockGranularity lockGranularity) public void setup() throws IOException { appenderatorsManager = new TestAppenderatorsManager(); - reportsFile = temporaryFolder.newFile( - StringUtils.format("IndexTaskTestReports-%s.json", System.currentTimeMillis()) - ); - final File deepStorageDir = temporaryFolder.newFolder(); final File cacheDir = temporaryFolder.newFolder(); segmentLoader = new SegmentLoaderLocalCacheManager( indexIO, From ae630575c45a1d499efeed644411a158945868c8 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 24 Jul 2019 20:31:20 -0700 Subject: [PATCH 17/24] Compile fix --- .../java/org/apache/druid/indexing/common/task/IndexTask.java | 1 - .../apache/druid/indexing/common/task/IngestionTestBase.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 798ad930e0d3..b89772c05f32 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -74,7 +74,6 @@ import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; -import org.apache.druid.segment.realtime.appenderator.Appenderators; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 15e9aa5e12ee..9c782a055567 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -24,7 +24,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClient; @@ -302,7 +302,7 @@ public ListenableFuture run(Task task) null, null, null, - new TaskReportFileWriter(taskReportsFile) + new SingleFileTaskReportFileWriter(taskReportsFile) ); if (task.isReady(box.getTaskActionClient())) { From 0d17365bad94ffe3267e342aa9e31293b07ce05b Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 26 Jul 2019 13:48:13 -0700 Subject: [PATCH 18/24] Address PR comments --- .../overlord/hrtr/HttpRemoteTaskRunner.java | 30 +--------- .../discovery/DruidNodeDiscoveryProvider.java | 2 +- ...egmentServerAnnouncerLifecycleHandler.java | 51 ++++++++-------- .../appenderator/AppenderatorImpl.java | 60 +++++++++++-------- .../UnifiedIndexerAppenderatorsManager.java | 1 - 5 files changed, 64 insertions(+), 80 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java index 0545a6e3aa9f..03ee844f518b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java @@ -41,7 +41,6 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; import org.apache.druid.discovery.WorkerNodeService; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; @@ -438,33 +437,11 @@ private void taskComplete( private void startWorkersHandling() throws InterruptedException { final CountDownLatch workerViewInitialized = new CountDownLatch(1); - DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER); - druidNodeDiscovery.registerListener( - new DruidNodeDiscovery.Listener() - { - @Override - public void nodesAdded(Collection nodes) - { - nodes.forEach(node -> addWorker(toWorker(node))); - } - - @Override - public void nodesRemoved(Collection nodes) - { - nodes.forEach(node -> removeWorker(toWorker(node))); - } - @Override - public void nodeViewInitialized() - { - //CountDownLatch.countDown() does nothing when count has already reached 0. - workerViewInitialized.countDown(); - } - } + DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForService( + WorkerNodeService.DISCOVERY_SERVICE_KEY ); - - DruidNodeDiscovery druidNodeDiscoveryIndexer = druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER); - druidNodeDiscoveryIndexer.registerListener( + druidNodeDiscovery.registerListener( new DruidNodeDiscovery.Listener() { @Override @@ -488,7 +465,6 @@ public void nodeViewInitialized() } ); - long workerDiscoveryStartTime = System.currentTimeMillis(); while (!workerViewInitialized.await(30, TimeUnit.SECONDS)) { if (System.currentTimeMillis() - workerDiscoveryStartTime > TimeUnit.MINUTES.toMillis(5)) { diff --git a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java index fc1cc8375ebc..ee9a2e8e2f73 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java @@ -42,7 +42,7 @@ public abstract class DruidNodeDiscoveryProvider private static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.BROKER, NodeType.HISTORICAL, NodeType.PEON, NodeType.INDEXER), DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.HISTORICAL, NodeType.PEON, NodeType.INDEXER), - WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.PEON, NodeType.INDEXER) + WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.MIDDLE_MANAGER, NodeType.INDEXER) ); private final ConcurrentHashMap serviceDiscoveryMap = diff --git a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java index 1f1636fc543c..e874a30c86d9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java @@ -21,6 +21,7 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; +import org.apache.druid.concurrent.LifecycleLock; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; @@ -42,10 +43,7 @@ public class CliIndexerDataSegmentServerAnnouncerLifecycleHandler private final DataSegmentServerAnnouncer dataSegmentServerAnnouncer; - // Synchronizes start/stop of this object. - private final Object startStopLock = new Object(); - - private volatile boolean started = false; + private final LifecycleLock lifecycleLock = new LifecycleLock(); @Inject public CliIndexerDataSegmentServerAnnouncerLifecycleHandler( @@ -58,8 +56,12 @@ public CliIndexerDataSegmentServerAnnouncerLifecycleHandler( @LifecycleStart public void start() throws IOException { - synchronized (startStopLock) { - if (started) { + if (!lifecycleLock.canStart()) { + throw new RuntimeException("Lifecycle lock could not start"); + } + + try { + if (lifecycleLock.isStarted()) { return; } @@ -71,35 +73,32 @@ public void start() throws IOException Throwables.propagateIfPossible(e, IOException.class); throw new RuntimeException(e); } - started = true; LOG.info("Started."); + lifecycleLock.started(); + } + finally { + lifecycleLock.exitStart(); } } @LifecycleStop public void stop() { - synchronized (startStopLock) { - if (!started) { - return; - } + if (!lifecycleLock.canStop()) { + throw new RuntimeException("Lifecycle lock could not stop"); + } - LOG.info("Stopping..."); - try { - dataSegmentServerAnnouncer.unannounce(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - started = false; - } - LOG.info("Stopped."); + if (!lifecycleLock.isStarted()) { + return; } - } - public boolean isStarted() - { - return started; + LOG.info("Stopping..."); + try { + dataSegmentServerAnnouncer.unannounce(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + LOG.info("Stopped."); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 980fbeec6264..4162a5f7640e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -171,35 +171,38 @@ public class AppenderatorImpl implements Appenderator CachePopulatorStats cachePopulatorStats ) { - this.schema = Preconditions.checkNotNull(schema, "schema"); - this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); - this.metrics = Preconditions.checkNotNull(metrics, "metrics"); - this.dataSegmentPusher = Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher"); - this.objectMapper = Preconditions.checkNotNull(objectMapper, "objectMapper"); - this.segmentAnnouncer = Preconditions.checkNotNull(segmentAnnouncer, "segmentAnnouncer"); - this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO"); - this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); - this.cache = cache; - this.sinkTimeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - this.texasRanger = conglomerate == null ? null : new SinkQuerySegmentWalker( - schema.getDataSource(), - sinkTimeline, + this( + schema, + tuningConfig, + metrics, + dataSegmentPusher, objectMapper, - emitter, - conglomerate, - queryExecutorService, - Preconditions.checkNotNull(cache, "cache"), - cacheConfig, - cachePopulatorStats + segmentAnnouncer, + conglomerate == null ? null : new SinkQuerySegmentWalker( + schema.getDataSource(), + new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ), + objectMapper, + emitter, + conglomerate, + queryExecutorService, + Preconditions.checkNotNull(cache, "cache"), + cacheConfig, + cachePopulatorStats + ), + indexIO, + indexMerger, + cache ); - maxBytesTuningConfig = TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()); log.info("Created Appenderator for dataSource[%s].", schema.getDataSource()); } /** - * This constructor allows the caller to provide its own sink timeline and segment walker. + * This constructor allows the caller to provide its own SinkQuerySegmentWalker. + * + * The sinkTimeline is set to the sink timeline of the provided SinkQuerySegmentWalker. + * If the SinkQuerySegmentWalker is null, a new sink timeline is initialized. * * It is used by UnifiedIndexerAppenderatorsManager which allows queries on data associated with multiple * Appenderators. @@ -211,7 +214,6 @@ public class AppenderatorImpl implements Appenderator DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, DataSegmentAnnouncer segmentAnnouncer, - VersionedIntervalTimeline sinkTimeline, SinkQuerySegmentWalker sinkQuerySegmentWalker, IndexIO indexIO, IndexMerger indexMerger, @@ -228,7 +230,15 @@ public class AppenderatorImpl implements Appenderator this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger"); this.cache = cache; this.texasRanger = sinkQuerySegmentWalker; - this.sinkTimeline = sinkTimeline; + + if (sinkQuerySegmentWalker == null) { + this.sinkTimeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); + } else { + this.sinkTimeline = sinkQuerySegmentWalker.getSinkTimeline(); + } + maxBytesTuningConfig = TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()); log.info("Created Appenderator for dataSource[%s].", schema.getDataSource()); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index a04714aa533d..4aa359370b0d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -126,7 +126,6 @@ public Appenderator createRealtimeAppenderatorForTask( dataSegmentPusher, objectMapper, segmentAnnouncer, - segmentWalker.getSinkTimeline(), segmentWalker, indexIO, indexMerger, From 3be1596a8d3845753b5e7ff400a2b6b339a79eb9 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 26 Jul 2019 16:51:08 -0700 Subject: [PATCH 19/24] Adjust TaskReportFileWriter and fix nits --- .../MultipleFileTaskReportFileWriter.java | 74 +++++++++++++++++++ .../SingleFileTaskReportFileWriter.java | 11 ++- .../indexing/common/TaskReportFileWriter.java | 46 +----------- .../overlord/BaseRestorableTaskRunner.java | 15 +--- .../overlord/ThreadingTaskRunner.java | 5 +- .../common/task/NoopTestTaskFileWriter.java | 12 +-- .../java/org/apache/druid/cli/CliIndexer.java | 3 +- .../java/org/apache/druid/cli/CliPeon.java | 2 +- 8 files changed, 101 insertions(+), 67 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/MultipleFileTaskReportFileWriter.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/MultipleFileTaskReportFileWriter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/MultipleFileTaskReportFileWriter.java new file mode 100644 index 000000000000..3e77f1f9bc23 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/MultipleFileTaskReportFileWriter.java @@ -0,0 +1,74 @@ +/* + * 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.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.commons.io.FileUtils; +import org.apache.druid.java.util.common.logger.Logger; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; + +public class MultipleFileTaskReportFileWriter implements TaskReportFileWriter +{ + private static final Logger log = new Logger(MultipleFileTaskReportFileWriter.class); + + private final Map taskReportFiles = new HashMap<>(); + + private ObjectMapper objectMapper; + + @Override + public void write(String taskId, Map reports) + { + final File reportsFile = taskReportFiles.get(taskId); + if (reportsFile == null) { + log.error("Could not find report file for task[%s]", taskId); + return; + } + + try { + final File reportsFileParent = reportsFile.getParentFile(); + if (reportsFileParent != null) { + FileUtils.forceMkdir(reportsFileParent); + } + objectMapper.writeValue(reportsFile, reports); + } + catch (Exception e) { + log.error(e, "Encountered exception in write()."); + } + } + + @Override + public void setObjectMapper(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } + + public void add(String taskId, File reportsFile) + { + taskReportFiles.put(taskId, reportsFile); + } + + public void delete(String taskId) + { + taskReportFiles.remove(taskId); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java index 3bd0c1a85f70..adf6ad580113 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/SingleFileTaskReportFileWriter.java @@ -19,17 +19,19 @@ package org.apache.druid.indexing.common; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.io.FileUtils; import org.apache.druid.java.util.common.logger.Logger; import java.io.File; import java.util.Map; -public class SingleFileTaskReportFileWriter extends TaskReportFileWriter +public class SingleFileTaskReportFileWriter implements TaskReportFileWriter { private static final Logger log = new Logger(SingleFileTaskReportFileWriter.class); private final File reportsFile; + private ObjectMapper objectMapper; public SingleFileTaskReportFileWriter(File reportsFile) { @@ -50,4 +52,11 @@ public void write(String taskId, Map reports) log.error(e, "Encountered exception in write()."); } } + + + @Override + public void setObjectMapper(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java index 360456642e80..908efe6e1a47 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskReportFileWriter.java @@ -20,52 +20,12 @@ package org.apache.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.commons.io.FileUtils; -import org.apache.druid.java.util.common.logger.Logger; -import java.io.File; -import java.util.HashMap; import java.util.Map; -public class TaskReportFileWriter +public interface TaskReportFileWriter { - private static final Logger log = new Logger(TaskReportFileWriter.class); + void write(String taskId, Map reports); - private final Map taskReportFiles = new HashMap<>(); - protected ObjectMapper objectMapper; - - public void add(String taskId, File reportsFile) - { - taskReportFiles.put(taskId, reportsFile); - } - - public void delete(String taskId) - { - taskReportFiles.remove(taskId); - } - - public void write(String taskId, Map reports) - { - final File reportsFile = taskReportFiles.get(taskId); - if (reportsFile == null) { - log.error("Could not find report file for task[%s]", taskId); - return; - } - - try { - final File reportsFileParent = reportsFile.getParentFile(); - if (reportsFileParent != null) { - FileUtils.forceMkdir(reportsFileParent); - } - objectMapper.writeValue(reportsFile, reports); - } - catch (Exception e) { - log.error(e, "Encountered exception in write()."); - } - } - - public void setObjectMapper(ObjectMapper objectMapper) - { - this.objectMapper = objectMapper; - } + void setObjectMapper(ObjectMapper objectMapper); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java index 15a83321f25a..4c9910f07974 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/BaseRestorableTaskRunner.java @@ -45,8 +45,7 @@ import java.util.concurrent.Executor; /** - * Base class for {@link ForkingTaskRunner} and {@link ThreadingTaskRunner} which support task rest - * oration. + * Base class for {@link ForkingTaskRunner} and {@link ThreadingTaskRunner} which support task restoration. */ public abstract class BaseRestorableTaskRunner implements TaskRunner { @@ -144,24 +143,12 @@ public void unregisterListener(String listenerId) } } - /** - * The concrete implementation should override this, as the implemention depends on the specifics of the - * work item implementation. - */ @Override public abstract Collection getRunningTasks(); - /** - * The concrete implementation should override this, as the implemention depends on the specifics of the - * work item implementation. - */ @Override public abstract Collection getPendingTasks(); - /** - * The concrete implementation should override this, as the implemention depends on the specifics of the - * work item implementation. - */ @Nullable @Override public abstract RunnerTaskState getRunnerTaskState(String taskId); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index a3e62eb1e459..112e62e22929 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -34,6 +34,7 @@ import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; @@ -93,7 +94,7 @@ public class ThreadingTaskRunner private final TaskLogPusher taskLogPusher; private final DruidNode node; private final AppenderatorsManager appenderatorsManager; - private final TaskReportFileWriter taskReportFileWriter; + private final MultipleFileTaskReportFileWriter taskReportFileWriter; private final ListeningExecutorService taskExecutor; private final ListeningExecutorService controlThreadExecutor; @@ -116,7 +117,7 @@ public ThreadingTaskRunner( this.taskLogPusher = taskLogPusher; this.node = node; this.appenderatorsManager = appenderatorsManager; - this.taskReportFileWriter = taskReportFileWriter; + this.taskReportFileWriter = (MultipleFileTaskReportFileWriter) taskReportFileWriter; this.taskExecutor = MoreExecutors.listeningDecorator( Execs.multiThreaded(workerConfig.getCapacity(), "threading-task-runner-executor-%d") ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java index 0fe9d7fcf1f2..467a1d7da7e6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java @@ -19,20 +19,22 @@ package org.apache.druid.indexing.common.task; -import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskReportFileWriter; import java.util.Map; -public class NoopTestTaskFileWriter extends SingleFileTaskReportFileWriter +public class NoopTestTaskFileWriter implements TaskReportFileWriter { - public NoopTestTaskFileWriter() + @Override + public void write(String id, Map reports) { - super(null); } @Override - public void write(String id, Map reports) + public void setObjectMapper(ObjectMapper objectMapper) { + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 3bd45cb6e59f..45756c64805d 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -43,6 +43,7 @@ import org.apache.druid.guice.QueryablePeonModule; import org.apache.druid.guice.annotations.RemoteChatHandler; import org.apache.druid.guice.annotations.Self; +import org.apache.druid.indexing.common.MultipleFileTaskReportFileWriter; import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.ThreadingTaskRunner; @@ -106,7 +107,7 @@ public void configure(Binder binder) CliPeon.bindTaskConfigAndClients(binder); - binder.bind(TaskReportFileWriter.class).toInstance(new TaskReportFileWriter()); + binder.bind(TaskReportFileWriter.class).toInstance(new MultipleFileTaskReportFileWriter()); binder.bind(TaskRunner.class).to(ThreadingTaskRunner.class); binder.bind(QuerySegmentWalker.class).to(ThreadingTaskRunner.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 4a6c2d8ace77..f1395857e4cc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -56,10 +56,10 @@ import org.apache.druid.guice.QueryablePeonModule; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; -import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.RemoteTaskActionClientFactory; From bec8f65b7bbfe46098e6b9dd9e33372a2ccb0d18 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 26 Jul 2019 17:08:31 -0700 Subject: [PATCH 20/24] Remove unnecessary closer --- .../overlord/ThreadingTaskRunner.java | 132 ++++++++---------- 1 file changed, 62 insertions(+), 70 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index 112e62e22929..5e2ffa63ff0a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -163,90 +163,82 @@ public TaskStatus call() ); try { - final Closer closer = Closer.create(); - try { - if (!attemptDir.mkdirs()) { - throw new IOE("Could not create directories: %s", attemptDir); - } - - final File taskFile = new File(taskDir, "task.json"); - final File reportsFile = new File(attemptDir, "report.json"); - taskReportFileWriter.add(task.getId(), reportsFile); + if (!attemptDir.mkdirs()) { + throw new IOE("Could not create directories: %s", attemptDir); + } - final ThreadingTaskRunnerWorkItem taskWorkItem; - // time to adjust process holders - synchronized (tasks) { - taskWorkItem = tasks.get(task.getId()); + final File taskFile = new File(taskDir, "task.json"); + final File reportsFile = new File(attemptDir, "report.json"); + taskReportFileWriter.add(task.getId(), reportsFile); - if (taskWorkItem == null) { - LOGGER.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); - throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); - } + final ThreadingTaskRunnerWorkItem taskWorkItem; + // time to adjust process holders + synchronized (tasks) { + taskWorkItem = tasks.get(task.getId()); - if (taskWorkItem.shutdown) { - throw new IllegalStateException("Task has been shut down!"); - } + if (taskWorkItem == null) { + LOGGER.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); + throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); } - if (!taskFile.exists()) { - jsonMapper.writeValue(taskFile, task); + if (taskWorkItem.shutdown) { + throw new IllegalStateException("Task has been shut down!"); } + } + + if (!taskFile.exists()) { + jsonMapper.writeValue(taskFile, task); + } - // This will block for a while. So we append the thread information with more details - final String priorThreadName = Thread.currentThread().getName(); - Thread.currentThread().setName(StringUtils.format("%s-[%s]", priorThreadName, task.getId())); + // This will block for a while. So we append the thread information with more details + final String priorThreadName = Thread.currentThread().getName(); + Thread.currentThread() + .setName(StringUtils.format("%s-[%s]", priorThreadName, task.getId())); - TaskStatus taskStatus = null; - final TaskToolbox toolbox = toolboxFactory.build(task); - try { - ListenableFuture taskStatusFuture = taskExecutor.submit( - new Callable() + TaskStatus taskStatus = null; + final TaskToolbox toolbox = toolboxFactory.build(task); + try { + ListenableFuture taskStatusFuture = taskExecutor.submit( + new Callable() + { + @Override + public TaskStatus call() { - @Override - public TaskStatus call() - { - taskWorkItem.setThread(Thread.currentThread()); - try { - return task.run(toolbox); - } - catch (Exception e) { - LOGGER.error(e, "Task[%s] exited with exception.", task.getId()); - return null; - } - finally { - taskWorkItem.setThread(null); - } + taskWorkItem.setThread(Thread.currentThread()); + try { + return task.run(toolbox); + } + catch (Exception e) { + LOGGER.error(e, "Task[%s] exited with exception.", task.getId()); + return null; + } + finally { + taskWorkItem.setThread(null); } } - ); - TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation); - TaskRunnerUtils.notifyStatusChanged( - listeners, - task.getId(), - TaskStatus.running(task.getId()) - ); - taskStatus = taskStatusFuture.get(); - } - finally { - taskWorkItem.setFinished(true); - Thread.currentThread().setName(priorThreadName); - if (reportsFile.exists()) { - taskLogPusher.pushTaskReports(task.getId(), reportsFile); - } - if (taskStatus == null) { - taskStatus = TaskStatus.failure(task.getId()); - } - } - - TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), taskStatus); - return taskStatus; - } - catch (Throwable t) { - throw closer.rethrow(t); + } + ); + TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation); + TaskRunnerUtils.notifyStatusChanged( + listeners, + task.getId(), + TaskStatus.running(task.getId()) + ); + taskStatus = taskStatusFuture.get(); } finally { - closer.close(); + taskWorkItem.setFinished(true); + Thread.currentThread().setName(priorThreadName); + if (reportsFile.exists()) { + taskLogPusher.pushTaskReports(task.getId(), reportsFile); + } + if (taskStatus == null) { + taskStatus = TaskStatus.failure(task.getId()); + } } + + TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), taskStatus); + return taskStatus; } catch (Throwable t) { LOGGER.info(t, "Exception caught during execution"); From 01c280fb7f20204603c722e0fb01ce8672911a36 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 26 Jul 2019 18:29:08 -0700 Subject: [PATCH 21/24] More PR comments --- .../overlord/ThreadingTaskRunner.java | 73 ++++++++----------- .../hrtr/HttpRemoteTaskRunnerTest.java | 28 ++----- .../initialization/ServerConfigSerdeTest.java | 59 +++++++++++++++ .../java/org/apache/druid/cli/CliPeon.java | 2 +- 4 files changed, 98 insertions(+), 64 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/initialization/ServerConfigSerdeTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index 5e2ffa63ff0a..c57fa891f008 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -47,7 +47,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; @@ -75,14 +74,13 @@ * TaskRunner implemention for the CliIndexer task execution service, which runs all tasks in a single process. * * Two thread pools are used: - * - A task execution pool, sized to number of worker slots. This is used to execute the Task run() methods. - * - A control thread pool, sized to worker slots * 2. The control threads setup and submit work to the - * task execution pool, and are also responsible for running graceful shutdown on the Task objects. - * Only one shutdown per-task can be running at a given time, and there is one control thread per task, - * thus the pool has 2 * worker slots. + * - A task execution pool, sized to number of worker slots. This is used to setup and execute the Task run() methods. + * - A control thread pool, sized to number of worker slots. The control threads are responsible for running graceful + * shutdown on the Task objects. Only one shutdown per-task can be running at a given time, + * so we allocate one control thread per worker slot. * - * Note that separate task logs are not supported, all task log entries will be written to the Indexer process log - * instead. + * Note that separate task logs are not currently supported, all task log entries will be written to the Indexer + * process log instead. */ public class ThreadingTaskRunner extends BaseRestorableTaskRunner @@ -122,7 +120,7 @@ public ThreadingTaskRunner( Execs.multiThreaded(workerConfig.getCapacity(), "threading-task-runner-executor-%d") ); this.controlThreadExecutor = MoreExecutors.listeningDecorator( - Execs.multiThreaded(workerConfig.getCapacity() * 2, "threading-task-runner-control-%d") + Execs.multiThreaded(workerConfig.getCapacity(), "threading-task-runner-control-%d") ); } @@ -147,7 +145,7 @@ public ListenableFuture run(Task task) task.getId(), k -> new ThreadingTaskRunnerWorkItem( task, - controlThreadExecutor.submit( + taskExecutor.submit( new Callable() { @Override public TaskStatus call() @@ -162,6 +160,8 @@ public TaskStatus call() node.getTlsPort() ); + final ThreadingTaskRunnerWorkItem taskWorkItem; + try { if (!attemptDir.mkdirs()) { throw new IOE("Could not create directories: %s", attemptDir); @@ -171,7 +171,6 @@ public TaskStatus call() final File reportsFile = new File(attemptDir, "report.json"); taskReportFileWriter.add(task.getId(), reportsFile); - final ThreadingTaskRunnerWorkItem taskWorkItem; // time to adjust process holders synchronized (tasks) { taskWorkItem = tasks.get(task.getId()); @@ -197,44 +196,30 @@ public TaskStatus call() TaskStatus taskStatus = null; final TaskToolbox toolbox = toolboxFactory.build(task); + TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation); + TaskRunnerUtils.notifyStatusChanged( + listeners, + task.getId(), + TaskStatus.running(task.getId()) + ); + + taskWorkItem.setThread(Thread.currentThread()); + try { - ListenableFuture taskStatusFuture = taskExecutor.submit( - new Callable() - { - @Override - public TaskStatus call() - { - taskWorkItem.setThread(Thread.currentThread()); - try { - return task.run(toolbox); - } - catch (Exception e) { - LOGGER.error(e, "Task[%s] exited with exception.", task.getId()); - return null; - } - finally { - taskWorkItem.setThread(null); - } - } - } - ); - TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation); - TaskRunnerUtils.notifyStatusChanged( - listeners, - task.getId(), - TaskStatus.running(task.getId()) - ); - taskStatus = taskStatusFuture.get(); + taskStatus = task.run(toolbox); + } + catch (Throwable t) { + LOGGER.info(t, "Exception caught while running the task."); } finally { taskWorkItem.setFinished(true); + if (taskStatus == null) { + taskStatus = TaskStatus.failure(task.getId()); + } Thread.currentThread().setName(priorThreadName); if (reportsFile.exists()) { taskLogPusher.pushTaskReports(task.getId(), reportsFile); } - if (taskStatus == null) { - taskStatus = TaskStatus.failure(task.getId()); - } } TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), taskStatus); @@ -250,7 +235,7 @@ public TaskStatus call() try { synchronized (tasks) { - final ThreadingTaskRunnerWorkItem taskWorkItem = tasks.remove(task.getId()); + tasks.remove(task.getId()); if (!stopping) { saveRunningTasks(); } @@ -272,6 +257,10 @@ public TaskStatus call() catch (Exception e) { LOGGER.error(e, "Suppressing exception caught while cleaning up task"); } + + // Make sure we clear the interrupt flag at the end, since this thread will be reused + // for other tasks. + Thread.interrupted(); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index eb4a76e2e3e0..a94dd5e9475d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -87,9 +87,7 @@ public void testFreshStart() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) - .andReturn(druidNodeDiscovery); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -180,9 +178,7 @@ public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) - .andReturn(druidNodeDiscovery); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -272,9 +268,7 @@ public void testTaskRunnerRestart() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) - .andReturn(druidNodeDiscovery); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -427,9 +421,7 @@ public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) - .andReturn(druidNodeDiscovery); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -605,9 +597,7 @@ public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) - .andReturn(druidNodeDiscovery); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -781,9 +771,7 @@ public void testMarkWorkersLazy() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) - .andReturn(druidNodeDiscovery); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -1218,9 +1206,7 @@ private HttpRemoteTaskRunner createTaskRunnerForTestTaskAddedOrUpdated( { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) - .andReturn(druidNodeDiscovery); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.INDEXER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); diff --git a/server/src/test/java/org/apache/druid/initialization/ServerConfigSerdeTest.java b/server/src/test/java/org/apache/druid/initialization/ServerConfigSerdeTest.java new file mode 100644 index 000000000000..a67899c04aa7 --- /dev/null +++ b/server/src/test/java/org/apache/druid/initialization/ServerConfigSerdeTest.java @@ -0,0 +1,59 @@ +/* + * 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.initialization; + +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.server.initialization.ServerConfig; +import org.junit.Assert; +import org.junit.Test; + +public class ServerConfigSerdeTest +{ + private static final DefaultObjectMapper objectMapper = new DefaultObjectMapper(); + + @Test + public void testSerde() throws Exception + { + ServerConfig defaultConfig = new ServerConfig(); + String defaultConfigJson = objectMapper.writeValueAsString(defaultConfig); + ServerConfig defaultConfig2 = objectMapper.readValue(defaultConfigJson, ServerConfig.class); + Assert.assertEquals(defaultConfig, defaultConfig2); + + ServerConfig modifiedConfig = new ServerConfig( + 999, + 888, + defaultConfig.isEnableRequestLimit(), + defaultConfig.getMaxIdleTime(), + defaultConfig.getDefaultQueryTimeout(), + defaultConfig.getMaxScatterGatherBytes(), + defaultConfig.getMaxQueryTimeout(), + defaultConfig.getMaxRequestHeaderSize(), + defaultConfig.getGracefulShutdownTimeout(), + defaultConfig.getUnannouncePropagationDelay(), + defaultConfig.getInflateBufferSize(), + defaultConfig.getCompressionLevel() + ); + String modifiedConfigJson = objectMapper.writeValueAsString(modifiedConfig); + ServerConfig modifiedConfig2 = objectMapper.readValue(modifiedConfigJson, ServerConfig.class); + Assert.assertEquals(modifiedConfig, modifiedConfig2); + Assert.assertEquals(999, modifiedConfig2.getNumThreads()); + Assert.assertEquals(888, modifiedConfig2.getQueueSize()); + } +} diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index f1395857e4cc..4a6c2d8ace77 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -56,10 +56,10 @@ import org.apache.druid.guice.QueryablePeonModule; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; +import org.apache.druid.indexing.common.TaskReportFileWriter; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.RemoteTaskActionClientFactory; From dd8f4ffd316be32ab3fde6a2419eccd7bae0b1c5 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 26 Jul 2019 18:43:08 -0700 Subject: [PATCH 22/24] Minor adjustments --- .../apache/druid/indexing/overlord/ThreadingTaskRunner.java | 6 +++--- .../org/apache/druid/indexing/common/TaskToolboxTest.java | 4 ++-- .../druid/indexing/common/task/CompactionTaskRunTest.java | 2 +- .../druid/indexing/common/task/CompactionTaskTest.java | 2 +- ...askFileWriter.java => NoopTestTaskReportFileWriter.java} | 2 +- .../druid/indexing/common/task/RealtimeIndexTaskTest.java | 2 +- .../parallel/AbstractParallelIndexSupervisorTaskTest.java | 4 ++-- .../apache/druid/indexing/overlord/TaskLifecycleTest.java | 4 ++-- .../apache/druid/indexing/worker/WorkerTaskManagerTest.java | 4 ++-- .../apache/druid/indexing/worker/WorkerTaskMonitorTest.java | 4 ++-- 10 files changed, 17 insertions(+), 17 deletions(-) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/{NoopTestTaskFileWriter.java => NoopTestTaskReportFileWriter.java} (94%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index c57fa891f008..6f2837abf5a2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -230,10 +230,10 @@ public TaskStatus call() throw new RuntimeException(t); } finally { - taskReportFileWriter.delete(task.getId()); - appenderatorsManager.removeAppenderatorForTask(task.getId()); - try { + taskReportFileWriter.delete(task.getId()); + appenderatorsManager.removeAppenderatorForTask(task.getId()); + synchronized (tasks) { tasks.remove(task.getId()); if (!stopping) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 0575afe585e3..36ad3a05f671 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -26,7 +26,7 @@ import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter; +import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -119,7 +119,7 @@ public void setUp() throws IOException null, null, null, - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 5f9bfcf2163e..cd00cf2a4fc4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -721,7 +721,7 @@ public List getLocations() null, null, null, - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index cae847706f2c..80eb89345aac 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1184,7 +1184,7 @@ private static class TestTaskToolbox extends TaskToolbox null, null, null, - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ); this.segmentFileMap = segmentFileMap; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskReportFileWriter.java similarity index 94% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskReportFileWriter.java index 467a1d7da7e6..0398a219516b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskFileWriter.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopTestTaskReportFileWriter.java @@ -25,7 +25,7 @@ import java.util.Map; -public class NoopTestTaskFileWriter implements TaskReportFileWriter +public class NoopTestTaskReportFileWriter implements TaskReportFileWriter { @Override public void write(String id, Map reports) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index a5009058834d..29a7391b5a1d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -999,7 +999,7 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 4323cd5cf1e9..95467d7904e7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -43,7 +43,7 @@ import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IngestionTestBase; -import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter; +import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; @@ -252,7 +252,7 @@ public File getStorageDirectory() null, null, null, - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 667c14398e7f..cb320e8bcfb6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -66,7 +66,7 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.KillTask; -import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter; +import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; @@ -628,7 +628,7 @@ public void unannounceSegments(Iterable segments) EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index b4c21b7e0df8..5ea1ffd8283b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -34,7 +34,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter; +import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.overlord.TestTaskRunner; @@ -131,7 +131,7 @@ public List getLocations() null, null, null, - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ), taskConfig, location diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 8c22aaa85a21..c59859706352 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -39,7 +39,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter; +import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.SingleTaskBackgroundRunner; import org.apache.druid.indexing.overlord.TestRemoteTaskRunnerConfig; @@ -179,7 +179,7 @@ private WorkerTaskMonitor createTaskMonitor() null, null, null, - new NoopTestTaskFileWriter() + new NoopTestTaskReportFileWriter() ), taskConfig, new NoopServiceEmitter(), From e3deef6bda9232635a16572e6275143b17aac2f5 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 29 Jul 2019 12:01:15 -0700 Subject: [PATCH 23/24] PR comments --- .../overlord/ThreadingTaskRunner.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index 6f2837abf5a2..c05fd1f37de2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -209,7 +209,7 @@ public TaskStatus call() taskStatus = task.run(toolbox); } catch (Throwable t) { - LOGGER.info(t, "Exception caught while running the task."); + LOGGER.error(t, "Exception caught while running the task."); } finally { taskWorkItem.setFinished(true); @@ -226,7 +226,7 @@ public TaskStatus call() return taskStatus; } catch (Throwable t) { - LOGGER.info(t, "Exception caught during execution"); + LOGGER.error(t, "Exception caught during execution"); throw new RuntimeException(t); } finally { @@ -312,7 +312,7 @@ private ListenableFuture scheduleTaskShutdown(ThreadingTaskRunnerWorkItem taskIn return taskInfo.shutdownFuture; } - ListenableFuture shutdownFuture = controlThreadExecutor.submit( + taskInfo.shutdownFuture = controlThreadExecutor.submit( new Callable() { @Override @@ -328,15 +328,17 @@ public Void call() ); if (status == null) { - if (taskInfo.thread != null) { - taskInfo.thread.interrupt(); + // Note that we can't truly force a hard termination of the task thread externally. + // In the future we may want to add a forceful shutdown method to the Task interface. + if (taskInfo.shutdownFuture != null) { + taskInfo.shutdownFuture.cancel(true); } } } catch (Exception e) { LOGGER.info(e, "Encountered exception while waiting for task [%s] shutdown", taskInfo.getTaskId()); - if (taskInfo.thread != null) { - taskInfo.thread.interrupt(); + if (taskInfo.shutdownFuture != null) { + taskInfo.shutdownFuture.cancel(true); } } return null; @@ -344,8 +346,7 @@ public Void call() } ); - taskInfo.shutdownFuture = shutdownFuture; - return shutdownFuture; + return taskInfo.shutdownFuture; } } From 6cf986a280cf264c11c1aae1023d244540170449 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 29 Jul 2019 13:41:48 -0700 Subject: [PATCH 24/24] ThreadingTaskRunner: cancel task run future not shutdownFuture and remove thread from workitem --- .../overlord/ThreadingTaskRunner.java | 92 +++++++------------ 1 file changed, 32 insertions(+), 60 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index c05fd1f37de2..c0bcaf4fb23b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -69,6 +69,7 @@ import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; /** * TaskRunner implemention for the CliIndexer task execution service, which runs all tasks in a single process. @@ -185,6 +186,7 @@ public TaskStatus call() } } + if (!taskFile.exists()) { jsonMapper.writeValue(taskFile, task); } @@ -203,8 +205,7 @@ public TaskStatus call() TaskStatus.running(task.getId()) ); - taskWorkItem.setThread(Thread.currentThread()); - + taskWorkItem.setState(RunnerTaskState.RUNNING); try { taskStatus = task.run(toolbox); } @@ -212,7 +213,7 @@ public TaskStatus call() LOGGER.error(t, "Exception caught while running the task."); } finally { - taskWorkItem.setFinished(true); + taskWorkItem.setState(RunnerTaskState.NONE); if (taskStatus == null) { taskStatus = TaskStatus.failure(task.getId()); } @@ -257,10 +258,6 @@ public TaskStatus call() catch (Exception e) { LOGGER.error(e, "Suppressing exception caught while cleaning up task"); } - - // Make sure we clear the interrupt flag at the end, since this thread will be reused - // for other tasks. - Thread.interrupted(); } } } @@ -322,18 +319,16 @@ public Void call() taskInfo.getTask().stopGracefully(taskConfig); try { - TaskStatus status = taskInfo.getResult().get( + taskInfo.getResult().get( taskConfig.getGracefulShutdownTimeout().toStandardDuration().getMillis(), TimeUnit.MILLISECONDS ); - - if (status == null) { - // Note that we can't truly force a hard termination of the task thread externally. - // In the future we may want to add a forceful shutdown method to the Task interface. - if (taskInfo.shutdownFuture != null) { - taskInfo.shutdownFuture.cancel(true); - } - } + } + catch (TimeoutException e) { + // Note that we can't truly force a hard termination of the task, interrupting the thread + // running the task to hopefully have it stop. + // In the future we may want to add a forceful shutdown method to the Task interface. + taskInfo.getResult().cancel(true); } catch (Exception e) { LOGGER.info(e, "Encountered exception while waiting for task [%s] shutdown", taskInfo.getTaskId()); @@ -419,29 +414,13 @@ public void stop() @Override public Collection getRunningTasks() { - synchronized (tasks) { - final List ret = new ArrayList<>(); - for (final ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) { - if (taskWorkItem.getThread() != null) { - ret.add(taskWorkItem); - } - } - return ret; - } + return getTasks(RunnerTaskState.RUNNING); } @Override public Collection getPendingTasks() { - synchronized (tasks) { - final List ret = new ArrayList<>(); - for (final ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) { - if (taskWorkItem.getThread() == null) { - ret.add(taskWorkItem); - } - } - return ret; - } + return getTasks(RunnerTaskState.PENDING); } @Nullable @@ -449,16 +428,19 @@ public Collection getPendingTasks() public RunnerTaskState getRunnerTaskState(String taskId) { final ThreadingTaskRunnerWorkItem workItem = tasks.get(taskId); - if (workItem == null) { - return null; - } else { - if (workItem.getThread() == null) { - return RunnerTaskState.PENDING; - } else if (!workItem.isFinished()) { - return RunnerTaskState.RUNNING; - } else { - return RunnerTaskState.NONE; + return workItem == null ? null : workItem.getState(); + } + + private Collection getTasks(RunnerTaskState state) + { + synchronized (tasks) { + final List ret = new ArrayList<>(); + for (final ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) { + if (taskWorkItem.getState() == state) { + ret.add(taskWorkItem); + } } + return ret; } } @@ -489,10 +471,9 @@ public QueryRunner getQueryRunnerForSegments( protected static class ThreadingTaskRunnerWorkItem extends TaskRunnerWorkItem { private final Task task; - private volatile Thread thread; - private boolean finished = false; private volatile boolean shutdown = false; - private ListenableFuture shutdownFuture; + private volatile ListenableFuture shutdownFuture; + private volatile RunnerTaskState state; private ThreadingTaskRunnerWorkItem( Task task, @@ -501,6 +482,7 @@ private ThreadingTaskRunnerWorkItem( { super(task.getId(), statusFuture); this.task = task; + this.state = RunnerTaskState.PENDING; } public Task getTask() @@ -526,24 +508,14 @@ public String getDataSource() return task.getDataSource(); } - public void setThread(Thread thread) - { - this.thread = thread; - } - - public Thread getThread() - { - return thread; - } - - public boolean isFinished() + public RunnerTaskState getState() { - return finished; + return state; } - public void setFinished(boolean finished) + public void setState(RunnerTaskState state) { - this.finished = finished; + this.state = state; } } }