Add CliIndexer process type and initial task runner implementation (#8107)

* Add CliIndexer process type and initial task runner implementation

* Fix HttpRemoteTaskRunnerTest

* Remove batch sanity check on PeonAppenderatorsManager

* Fix paralle index tests

* PR comments

* Adjust Jersey resource logging

* Additional cleanup

* Fix SystemSchemaTest

* Add comment to LocalDataSegmentPusherTest absolute path test

* More PR comments

* Use Server annotated with RemoteChatHandler

* More PR comments

* Checkstyle

* PR comments

* Add task shutdown to stopGracefully

* Small cleanup

* Compile fix

* Address PR comments

* Adjust TaskReportFileWriter and fix nits

* Remove unnecessary closer

* More PR comments

* Minor adjustments

* PR comments

* ThreadingTaskRunner: cancel  task run future not shutdownFuture and remove thread from workitem
This commit is contained in:
Jonathan Wei 2019-07-29 17:06:33 -07:00 committed by Himanshu
parent cc4450db12
commit 640b7afc1c
79 changed files with 3018 additions and 512 deletions

View File

@ -24,14 +24,18 @@ import com.google.inject.TypeLiteral;
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<Class<?>>(){}, JSR311Resource.class)
.addBinding()
.toInstance(resourceClazz);

View File

@ -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. This limit is applied to prevent lookup handling from consuming too many server HTTP connections.
# API for configuring lookups
## Bulk update

View File

@ -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;

View File

@ -37,6 +37,7 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
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;
@ -72,6 +73,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
Optional<ChatHandlerProvider> chatHandlerProvider,
CircularBuffer<Throwable> savedParseExceptions,
RowIngestionMetersFactory rowIngestionMetersFactory,
AppenderatorsManager appenderatorsManager,
LockGranularity lockGranularityToUse
)
{
@ -82,6 +84,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
chatHandlerProvider,
savedParseExceptions,
rowIngestionMetersFactory,
appenderatorsManager,
lockGranularityToUse
);
this.task = task;

View File

@ -29,6 +29,7 @@ import org.apache.druid.indexing.common.task.TaskResource;
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 class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long>
@JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject AuthorizerMapper authorizerMapper,
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
@JacksonInject ObjectMapper configMapper
@JacksonInject ObjectMapper configMapper,
@JacksonInject AppenderatorsManager appenderatorsManager
)
{
super(
@ -76,7 +78,8 @@ public class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long>
chatHandlerProvider,
authorizerMapper,
rowIngestionMetersFactory,
getFormattedGroupId(dataSchema.getDataSource(), TYPE)
getFormattedGroupId(dataSchema.getDataSource(), TYPE),
appenderatorsManager
);
this.configMapper = configMapper;
this.ioConfig = ioConfig;
@ -136,6 +139,7 @@ public class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long>
chatHandlerProvider,
savedParseExceptions,
rowIngestionMetersFactory,
appenderatorsManager,
lockGranularityToUse
);
}

View File

@ -258,7 +258,8 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
null,
null,
rowIngestionMetersFactory,
sortingMapper
sortingMapper,
null
));
}
return taskList;

View File

@ -51,9 +51,9 @@ import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.LockGranularity;
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;
@ -68,6 +68,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.Tasks;
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;
@ -142,6 +143,7 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
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;
@ -230,6 +232,7 @@ public class KafkaIndexTaskTest
private Long maxTotalRows = null;
private Period intermediateHandoffPeriod = null;
private AppenderatorsManager appenderatorsManager;
private TaskToolboxFactory toolboxFactory;
private IndexerMetadataStorageCoordinator metadataStorageCoordinator;
private TaskStorage taskStorage;
@ -372,6 +375,7 @@ public class KafkaIndexTaskTest
topic = getTopicName();
records = generateRecords(topic);
reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json");
appenderatorsManager = new TestAppenderatorsManager();
makeToolboxFactory();
}
@ -2534,7 +2538,8 @@ public class KafkaIndexTaskTest
null,
null,
rowIngestionMetersFactory,
OBJECT_MAPPER
OBJECT_MAPPER,
appenderatorsManager
);
task.setPollRetryMs(POLL_RETRY_MS);
return task;
@ -2701,6 +2706,7 @@ public class KafkaIndexTaskTest
final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig();
dataSegmentPusherConfig.storageDirectory = getSegmentDirectory();
final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig);
toolboxFactory = new TaskToolboxFactory(
taskConfig,
taskActionClientFactory,
@ -2726,7 +2732,7 @@ public class KafkaIndexTaskTest
EasyMock.createNiceMock(DruidNode.class),
new LookupNodeService("tier"),
new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0),
new TaskReportFileWriter(reportsFile)
new SingleFileTaskReportFileWriter(reportsFile)
);
}

View File

@ -82,6 +82,7 @@ import org.apache.druid.segment.indexing.DataSchema;
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;
@ -3556,7 +3557,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
null,
null,
rowIngestionMetersFactory,
objectMapper
objectMapper,
new DummyForInjectionAppenderatorsManager()
);
}

View File

@ -28,6 +28,7 @@ import org.apache.druid.indexing.common.task.TaskResource;
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 class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
@JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject AuthorizerMapper authorizerMapper,
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
@JacksonInject AWSCredentialsConfig awsCredentialsConfig
@JacksonInject AWSCredentialsConfig awsCredentialsConfig,
@JacksonInject AppenderatorsManager appenderatorsManager
)
{
super(
@ -63,7 +65,8 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
chatHandlerProvider,
authorizerMapper,
rowIngestionMetersFactory,
getFormattedGroupId(dataSchema.getDataSource(), TYPE)
getFormattedGroupId(dataSchema.getDataSource(), TYPE),
appenderatorsManager
);
this.awsCredentialsConfig = awsCredentialsConfig;
}
@ -79,6 +82,7 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
chatHandlerProvider,
savedParseExceptions,
rowIngestionMetersFactory,
appenderatorsManager,
lockGranularityToUse
);
}

View File

@ -37,6 +37,7 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
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;
@ -66,6 +67,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
Optional<ChatHandlerProvider> chatHandlerProvider,
CircularBuffer<Throwable> savedParseExceptions,
RowIngestionMetersFactory rowIngestionMetersFactory,
AppenderatorsManager appenderatorsManager,
LockGranularity lockGranularityToUse
)
{
@ -76,6 +78,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
chatHandlerProvider,
savedParseExceptions,
rowIngestionMetersFactory,
appenderatorsManager,
lockGranularityToUse
);
this.task = task;

View File

@ -173,7 +173,8 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
null,
null,
rowIngestionMetersFactory,
awsCredentialsConfig
awsCredentialsConfig,
null
));
}
return taskList;

View File

@ -57,9 +57,9 @@ import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.LockGranularity;
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;
@ -75,6 +75,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.Tasks;
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;
@ -141,6 +142,7 @@ import org.apache.druid.segment.loading.DataSegmentPusher;
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;
@ -231,6 +233,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;
@ -316,6 +319,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
recordSupplier = mock(KinesisRecordSupplier.class);
appenderatorsManager = new TestAppenderatorsManager();
// sleep required because of kinesalite
Thread.sleep(500);
makeToolboxFactory();
@ -2720,7 +2725,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
null,
null,
rowIngestionMetersFactory,
null
null,
appenderatorsManager
);
}
@ -2880,6 +2886,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
final LocalDataSegmentPusherConfig dataSegmentPusherConfig = new LocalDataSegmentPusherConfig();
dataSegmentPusherConfig.storageDirectory = getSegmentDirectory();
final DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher(dataSegmentPusherConfig);
toolboxFactory = new TaskToolboxFactory(
taskConfig,
taskActionClientFactory,
@ -2905,7 +2912,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
EasyMock.createNiceMock(DruidNode.class),
new LookupNodeService("tier"),
new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0),
new TaskReportFileWriter(reportsFile)
new SingleFileTaskReportFileWriter(reportsFile)
);
}
@ -3088,7 +3095,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
@JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject AuthorizerMapper authorizerMapper,
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
@JacksonInject AWSCredentialsConfig awsCredentialsConfig
@JacksonInject AWSCredentialsConfig awsCredentialsConfig,
@JacksonInject AppenderatorsManager appenderatorsManager
)
{
super(
@ -3101,7 +3109,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
chatHandlerProvider,
authorizerMapper,
rowIngestionMetersFactory,
awsCredentialsConfig
awsCredentialsConfig,
appenderatorsManager
);
}

View File

@ -80,6 +80,7 @@ import org.apache.druid.segment.indexing.DataSchema;
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;
@ -4259,7 +4260,8 @@ public class KinesisSupervisorTest extends EasyMockSupport
null,
null,
rowIngestionMetersFactory,
null
null,
new DummyForInjectionAppenderatorsManager()
);
}

View File

@ -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<String, File> taskReportFiles = new HashMap<>();
private ObjectMapper objectMapper;
@Override
public void write(String taskId, Map<String, TaskReport> 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);
}
}

View File

@ -0,0 +1,62 @@
/*
* 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.Map;
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)
{
this.reportsFile = reportsFile;
}
@Override
public void write(String taskId, Map<String, TaskReport> 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().");
}
}
@Override
public void setObjectMapper(ObjectMapper objectMapper)
{
this.objectMapper = objectMapper;
}
}

View File

@ -20,40 +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.Map;
public class TaskReportFileWriter
public interface TaskReportFileWriter
{
private static final Logger log = new Logger(TaskReportFileWriter.class);
void write(String taskId, Map<String, TaskReport> reports);
private final File reportsFile;
private ObjectMapper objectMapper;
public TaskReportFileWriter(File reportFile)
{
this.reportsFile = reportFile;
}
public void write(Map<String, TaskReport> 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().");
}
}
public void setObjectMapper(ObjectMapper objectMapper)
{
this.objectMapper = objectMapper;
}
void setObjectMapper(ObjectMapper objectMapper);
}

View File

@ -76,7 +76,7 @@ import org.apache.druid.segment.realtime.FireDepartment;
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;
@ -175,6 +175,9 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
@JsonIgnore
private String errorMsg;
@JsonIgnore
private AppenderatorsManager appenderatorsManager;
@JsonCreator
public AppenderatorDriverRealtimeIndexTask(
@JsonProperty("id") String id,
@ -183,7 +186,8 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
@JsonProperty("context") Map<String, Object> context,
@JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject AuthorizerMapper authorizerMapper,
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
@JacksonInject AppenderatorsManager appenderatorsManager
)
{
super(
@ -204,6 +208,7 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
this.ingestionState = IngestionState.NOT_STARTED;
this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
this.appenderatorsManager = appenderatorsManager;
this.lockGranularity = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK)
? LockGranularity.TIME_CHUNK
: LockGranularity.SEGMENT;
@ -282,9 +287,10 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
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(
segmentId -> {
@ -415,7 +421,7 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
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
@ -432,12 +438,14 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
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());
}
@ -473,6 +481,14 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
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();
}
}
}
}
@ -728,14 +744,15 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
);
}
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,

View File

@ -74,6 +74,7 @@ import org.apache.druid.segment.indexing.DataSchema;
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;
@ -145,6 +146,9 @@ public class CompactionTask extends AbstractBatchIndexTask
@JsonIgnore
private List<IndexTask> indexTaskSpecs;
@JsonIgnore
private AppenderatorsManager appenderatorsManager;
@JsonCreator
public CompactionTask(
@JsonProperty("id") final String id,
@ -165,7 +169,8 @@ public class CompactionTask extends AbstractBatchIndexTask
@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);
@ -192,6 +197,7 @@ public class CompactionTask extends AbstractBatchIndexTask
this.coordinatorClient = coordinatorClient;
this.segmentLoaderFactory = segmentLoaderFactory;
this.retryPolicyFactory = retryPolicyFactory;
this.appenderatorsManager = appenderatorsManager;
}
@JsonProperty
@ -307,7 +313,9 @@ public class CompactionTask extends AbstractBatchIndexTask
getContext(),
authorizerMapper,
chatHandlerProvider,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
))
.collect(Collectors.toList());
}
@ -909,6 +917,7 @@ public class CompactionTask extends AbstractBatchIndexTask
private final CoordinatorClient coordinatorClient;
private final SegmentLoaderFactory segmentLoaderFactory;
private final RetryPolicyFactory retryPolicyFactory;
private final AppenderatorsManager appenderatorsManager;
@Nullable
private Interval interval;
@ -935,7 +944,8 @@ public class CompactionTask extends AbstractBatchIndexTask
RowIngestionMetersFactory rowIngestionMetersFactory,
CoordinatorClient coordinatorClient,
SegmentLoaderFactory segmentLoaderFactory,
RetryPolicyFactory retryPolicyFactory
RetryPolicyFactory retryPolicyFactory,
AppenderatorsManager appenderatorsManager
)
{
this.dataSource = dataSource;
@ -946,6 +956,7 @@ public class CompactionTask extends AbstractBatchIndexTask
this.coordinatorClient = coordinatorClient;
this.segmentLoaderFactory = segmentLoaderFactory;
this.retryPolicyFactory = retryPolicyFactory;
this.appenderatorsManager = appenderatorsManager;
}
public Builder interval(Interval interval)
@ -1017,7 +1028,8 @@ public class CompactionTask extends AbstractBatchIndexTask
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
}
}

View File

@ -128,6 +128,13 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
@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.
@ -265,6 +272,14 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
@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()) {
@ -288,7 +303,7 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
}
errorMsg = Throwables.getStackTraceAsString(effectiveException);
toolbox.getTaskReportFileWriter().write(getTaskCompletionReports());
toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports());
return TaskStatus.failure(
getId(),
errorMsg
@ -352,7 +367,7 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
indexerSchema = determineConfigStatus.getSchema();
if (indexerSchema == null) {
errorMsg = determineConfigStatus.getErrorMsg();
toolbox.getTaskReportFileWriter().write(getTaskCompletionReports());
toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports());
return TaskStatus.failure(
getId(),
errorMsg
@ -399,7 +414,7 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
specVersion,
version
);
toolbox.getTaskReportFileWriter().write(null);
toolbox.getTaskReportFileWriter().write(getId(), null);
return TaskStatus.failure(getId());
}
}
@ -438,14 +453,14 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
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
@ -463,6 +478,13 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
@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();
@ -497,9 +519,9 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
}
finally {
Thread.currentThread().setContextClassLoader(oldLoader);
runThread.interrupt();
}
}
}
@GET

View File

@ -48,6 +48,7 @@ import org.apache.druid.indexing.common.TaskReport;
import org.apache.druid.indexing.common.TaskToolbox;
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;
@ -73,7 +74,7 @@ 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.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;
@ -171,6 +172,18 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@JsonIgnore
private String errorMsg;
@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,
@ -179,7 +192,8 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@JsonProperty("context") final Map<String, Object> context,
@JacksonInject AuthorizerMapper authorizerMapper,
@JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory,
@JacksonInject AppenderatorsManager appenderatorsManager
)
{
this(
@ -191,7 +205,8 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
context,
authorizerMapper,
chatHandlerProvider,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
}
@ -204,7 +219,8 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
Map<String, Object> context,
AuthorizerMapper authorizerMapper,
ChatHandlerProvider chatHandlerProvider,
RowIngestionMetersFactory rowIngestionMetersFactory
RowIngestionMetersFactory rowIngestionMetersFactory,
AppenderatorsManager appenderatorsManager
)
{
super(
@ -229,6 +245,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
this.ingestionState = IngestionState.NOT_STARTED;
this.determinePartitionsMeters = rowIngestionMetersFactory.createRowIngestionMeters();
this.buildSegmentsMeters = rowIngestionMetersFactory.createRowIngestionMeters();
this.appenderatorsManager = appenderatorsManager;
}
@Override
@ -403,6 +420,14 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@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());
@ -478,7 +503,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
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
@ -492,6 +517,23 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
}
}
@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<String, TaskReport> getTaskCompletionReports()
{
return TaskReport.buildTaskReports(
@ -882,6 +924,8 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator);
final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir)
) {
this.appenderator = appenderator;
driver.startJob();
while (firehose.hasMore()) {
@ -950,7 +994,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
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
@ -964,7 +1008,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
);
log.info("Published segments: %s", Lists.transform(published.getSegments(), DataSegment::getId));
toolbox.getTaskReportFileWriter().write(getTaskCompletionReports());
toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports());
return TaskStatus.success(getId());
}
}
@ -1046,14 +1090,15 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
}
}
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,

View File

@ -518,6 +518,14 @@ public class RealtimeIndexTask extends AbstractTask
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();
}
}
}
}

View File

@ -37,6 +37,7 @@ import org.apache.druid.indexing.common.TaskToolbox;
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.AbstractBatchIndexTask;
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
import org.apache.druid.indexing.common.task.IndexTask;
@ -63,7 +64,7 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics;
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;
@ -105,6 +106,11 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
private final String supervisorTaskId;
private final IndexingServiceClient indexingServiceClient;
private final IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory;
private final AppenderatorsManager appenderatorsManager;
private Appenderator appenderator;
private Thread runThread;
private boolean stopped = false;
@JsonCreator
public ParallelIndexSubTask(
@ -117,7 +123,8 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
@JsonProperty("spec") final ParallelIndexIngestionSpec ingestionSchema,
@JsonProperty("context") final Map<String, Object> context,
@JacksonInject IndexingServiceClient indexingServiceClient,
@JacksonInject IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory
@JacksonInject IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory,
@JacksonInject AppenderatorsManager appenderatorsManager
)
{
super(
@ -137,6 +144,7 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
this.supervisorTaskId = supervisorTaskId;
this.indexingServiceClient = indexingServiceClient;
this.taskClientFactory = taskClientFactory;
this.appenderatorsManager = appenderatorsManager;
}
@Override
@ -189,6 +197,14 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
@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();
@ -399,6 +415,7 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator);
final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir)
) {
this.appenderator = appenderator;
driver.startJob();
final Set<DataSegment> pushedSegments = new HashSet<>();
@ -468,6 +485,7 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
}
}
private static Granularity findSegmentGranularity(GranularitySpec granularitySpec)
{
if (granularitySpec instanceof UniformGranularitySpec) {
@ -477,14 +495,15 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
}
}
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,
@ -508,4 +527,18 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask
toolbox.getDataSegmentKiller()
);
}
@Override
public void stopGracefully(TaskConfig taskConfig)
{
synchronized (this) {
stopped = true;
if (appenderator != null) {
appenderator.closeNow();
}
if (runThread != null) {
runThread.interrupt();
}
}
}
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.indexing.common.task.batch.parallel;
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 @@ class ParallelIndexSubTaskSpec extends SubTaskSpec<ParallelIndexSubTask>
getIngestionSpec(),
getContext(),
null,
null
null,
new DummyForInjectionAppenderatorsManager()
);
}
}

View File

@ -55,6 +55,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.indexing.TuningConfig;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
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;
@ -106,10 +107,14 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
private final ChatHandlerProvider chatHandlerProvider;
private final AuthorizerMapper authorizerMapper;
private final RowIngestionMetersFactory rowIngestionMetersFactory;
private final AppenderatorsManager appenderatorsManager;
private final ConcurrentHashMap<Interval, AtomicInteger> 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;
@ -123,7 +128,8 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
@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(
@ -146,6 +152,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
this.chatHandlerProvider = chatHandlerProvider;
this.authorizerMapper = authorizerMapper;
this.rowIngestionMetersFactory = rowIngestionMetersFactory;
this.appenderatorsManager = appenderatorsManager;
if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions()
!= TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS) {
@ -271,8 +278,13 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
@Override
public void stopGracefully(TaskConfig taskConfig)
{
synchronized (this) {
stopped = true;
}
if (runner != null) {
runner.stopGracefully();
} else if (sequentialIndexTask != null) {
sequentialIndexTask.stopGracefully(taskConfig);
}
}
@ -328,29 +340,40 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
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) throws Exception
{
final IndexTask indexTask = new IndexTask(
getId(),
getGroupId(),
getTaskResource(),
getDataSource(),
new IndexIngestionSpec(
getIngestionSchema().getDataSchema(),
getIngestionSchema().getIOConfig(),
convertToIndexTuningConfig(getIngestionSchema().getTuningConfig())
),
getContext(),
authorizerMapper,
chatHandlerProvider,
rowIngestionMetersFactory
);
if (indexTask.isReady(toolbox.getTaskActionClient())) {
return indexTask.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
);
}
if (sequentialIndexTask.isReady(toolbox.getTaskActionClient())) {
return sequentialIndexTask.run(toolbox);
} else {
return TaskStatus.failure(getId());
}

View File

@ -0,0 +1,209 @@
/*
* 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.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<WorkItemType extends TaskRunnerWorkItem> implements TaskRunner
{
protected static final EmittingLogger LOG = new EmittingLogger(BaseRestorableTaskRunner.class);
protected static final String TASK_RESTORE_FILENAME = "restore.json";
protected final CopyOnWriteArrayList<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
/** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */
protected final ConcurrentHashMap<String, WorkItemType> 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<Pair<Task, ListenableFuture<TaskStatus>>> 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<Pair<Task, ListenableFuture<TaskStatus>>> 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<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listener.getListenerId())) {
throw new ISE("Listener [%s] already registered", listener.getListenerId());
}
}
final Pair<TaskRunnerListener, Executor> 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<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listenerId)) {
listeners.remove(pair);
LOG.info("Unregistered listener [%s]", listenerId);
return;
}
}
}
@Override
public abstract Collection<TaskRunnerWorkItem> getRunningTasks();
@Override
public abstract Collection<TaskRunnerWorkItem> getPendingTasks();
@Nullable
@Override
public abstract RunnerTaskState getRunnerTaskState(String taskId);
@Override
public Collection<TaskRunnerWorkItem> 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<String> 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<String> runningTasks;
@JsonCreator
public TaskRestoreInfo(
@JsonProperty("runningTasks") List<String> runningTasks
)
{
this.runningTasks = runningTasks;
}
public List<String> getRunningTasks()
{
return runningTasks;
}
}
}

View File

@ -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.base.Splitter;
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.io.Files;
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.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;
@ -83,31 +78,23 @@ import java.util.Properties;
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<ForkingTaskRunner.ForkingTaskRunnerWorkItem>
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<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>();
/** Writes must be synchronized. This is only a ConcurrentMap so "informational" reads can occur without waiting. */
private final ConcurrentHashMap<String, ForkingTaskRunnerWorkItem> tasks = new ConcurrentHashMap<>();
private volatile boolean stopping = false;
@ -122,11 +109,10 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
@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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
);
}
@Override
public List<Pair<Task, ListenableFuture<TaskStatus>>> 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<Pair<Task, ListenableFuture<TaskStatus>>> 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<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listener.getListenerId())) {
throw new ISE("Listener [%s] already registered", listener.getListenerId());
}
}
final Pair<TaskRunnerListener, Executor> 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<TaskRunnerListener, Executor> pair : listeners) {
if (pair.lhs.getListenerId().equals(listenerId)) {
listeners.remove(pair);
log.info("Unregistered listener [%s]", listenerId);
return;
}
}
}
@Override
public ListenableFuture<TaskStatus> run(final Task task)
{
@ -257,19 +168,19 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
synchronized (tasks) {
final ForkingTaskRunnerWorkItem taskWorkItem = tasks.get(task.getId());
if (taskWorkItem == null) {
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 == null) {
log.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit();
throw new ISE("TaskInfo disappeared for task[%s]!", task.getId());
}
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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
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 class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
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");
}
}
}
@ -535,25 +446,25 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
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<String> 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)
@ -561,25 +472,25 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
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;
}
@ -617,14 +528,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
@Override
public Collection<TaskRunnerWorkItem> getKnownTasks()
{
synchronized (tasks) {
return Lists.newArrayList(tasks.values());
}
}
@Nullable
@Override
public RunnerTaskState getRunnerTaskState(String taskId)
@ -681,28 +584,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
);
}
/**
* 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<String> 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);
}
}
/**
* Close task output stream (input stream of process) sending EOF telling process to terminate, destroying the process
* if an exception is encountered.
@ -711,42 +592,18 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
{
if (taskInfo.processHolder != null) {
// Will trigger normal failure mechanisms due to process exit
log.info("Closing output stream to task[%s].", taskInfo.getTask().getId());
LOGGER.info("Closing output stream to task[%s].", taskInfo.getTask().getId());
try {
taskInfo.processHolder.process.getOutputStream().close();
}
catch (Exception e) {
log.warn(e, "Failed to close stdout to task[%s]. Destroying task.", taskInfo.getTask().getId());
LOGGER.warn(e, "Failed to close stdout to task[%s]. Destroying task.", taskInfo.getTask().getId());
taskInfo.processHolder.process.destroy();
}
}
}
private File getRestoreFile()
{
return new File(taskConfig.getBaseTaskDir(), TASK_RESTORE_FILENAME);
}
private static class TaskRestoreInfo
{
@JsonProperty
private final List<String> runningTasks;
@JsonCreator
public TaskRestoreInfo(
@JsonProperty("runningTasks") List<String> runningTasks
)
{
this.runningTasks = runningTasks;
}
public List<String> getRunningTasks()
{
return runningTasks;
}
}
private static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem
protected static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem
{
private final Task task;

View File

@ -0,0 +1,521 @@
/*
* 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.databind.ObjectMapper;
import com.google.common.base.Joiner;
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableSet;
import com.google.common.io.ByteSource;
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.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.MultipleFileTaskReportFileWriter;
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.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
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.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
* 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 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 currently supported, all task log entries will be written to the Indexer
* process log instead.
*/
public class ThreadingTaskRunner
extends BaseRestorableTaskRunner<ThreadingTaskRunner.ThreadingTaskRunnerWorkItem>
implements TaskLogStreamer, QuerySegmentWalker
{
private static final EmittingLogger LOGGER = new EmittingLogger(ThreadingTaskRunner.class);
private final TaskToolboxFactory toolboxFactory;
private final TaskLogPusher taskLogPusher;
private final DruidNode node;
private final AppenderatorsManager appenderatorsManager;
private final MultipleFileTaskReportFileWriter taskReportFileWriter;
private final ListeningExecutorService taskExecutor;
private final ListeningExecutorService controlThreadExecutor;
private volatile boolean stopping = false;
@Inject
public ThreadingTaskRunner(
TaskToolboxFactory toolboxFactory,
TaskConfig taskConfig,
WorkerConfig workerConfig,
TaskLogPusher taskLogPusher,
ObjectMapper jsonMapper,
AppenderatorsManager appenderatorsManager,
TaskReportFileWriter taskReportFileWriter,
@Self DruidNode node
)
{
super(jsonMapper, taskConfig);
this.toolboxFactory = toolboxFactory;
this.taskLogPusher = taskLogPusher;
this.node = node;
this.appenderatorsManager = appenderatorsManager;
this.taskReportFileWriter = (MultipleFileTaskReportFileWriter) taskReportFileWriter;
this.taskExecutor = MoreExecutors.listeningDecorator(
Execs.multiThreaded(workerConfig.getCapacity(), "threading-task-runner-executor-%d")
);
this.controlThreadExecutor = MoreExecutors.listeningDecorator(
Execs.multiThreaded(workerConfig.getCapacity(), "threading-task-runner-control-%d")
);
}
@Override
public Optional<ByteSource> 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 void start()
{
// Nothing to start.
}
@Override
public ListenableFuture<TaskStatus> run(Task task)
{
synchronized (tasks) {
tasks.computeIfAbsent(
task.getId(), k ->
new ThreadingTaskRunnerWorkItem(
task,
taskExecutor.submit(
new Callable<TaskStatus>() {
@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()
);
final ThreadingTaskRunnerWorkItem taskWorkItem;
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);
// time to adjust process holders
synchronized (tasks) {
taskWorkItem = tasks.get(task.getId());
if (taskWorkItem == null) {
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()) {
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);
TaskRunnerUtils.notifyLocationChanged(listeners, task.getId(), taskLocation);
TaskRunnerUtils.notifyStatusChanged(
listeners,
task.getId(),
TaskStatus.running(task.getId())
);
taskWorkItem.setState(RunnerTaskState.RUNNING);
try {
taskStatus = task.run(toolbox);
}
catch (Throwable t) {
LOGGER.error(t, "Exception caught while running the task.");
}
finally {
taskWorkItem.setState(RunnerTaskState.NONE);
if (taskStatus == null) {
taskStatus = TaskStatus.failure(task.getId());
}
Thread.currentThread().setName(priorThreadName);
if (reportsFile.exists()) {
taskLogPusher.pushTaskReports(task.getId(), reportsFile);
}
}
TaskRunnerUtils.notifyStatusChanged(listeners, task.getId(), taskStatus);
return taskStatus;
}
catch (Throwable t) {
LOGGER.error(t, "Exception caught during execution");
throw new RuntimeException(t);
}
finally {
try {
taskReportFileWriter.delete(task.getId());
appenderatorsManager.removeAppenderatorForTask(task.getId());
synchronized (tasks) {
tasks.remove(task.getId());
if (!stopping) {
saveRunningTasks();
}
}
try {
if (!stopping && taskDir.exists()) {
LOGGER.info("Removing task directory: %s", taskDir);
FileUtils.deleteDirectory(taskDir);
}
}
catch (Exception e) {
LOGGER.makeAlert(e, "Failed to delete task directory")
.addData("taskDir", taskDir.toString())
.addData("task", task.getId())
.emit();
}
}
catch (Exception e) {
LOGGER.error(e, "Suppressing exception caught while cleaning up task");
}
}
}
}
)
)
);
saveRunningTasks();
return tasks.get(task.getId()).getResult();
}
}
@Override
public void shutdown(String taskid, String reason)
{
LOGGER.info("Shutdown [%s] because: [%s]", taskid, reason);
final ThreadingTaskRunnerWorkItem taskInfo;
synchronized (tasks) {
taskInfo = tasks.get(taskid);
if (taskInfo == null) {
LOGGER.info("Ignoring request to cancel unknown task: %s", taskid);
return;
}
if (taskInfo.shutdown) {
LOGGER.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;
}
taskInfo.shutdownFuture = controlThreadExecutor.submit(
new Callable<Void>()
{
@Override
public Void call()
{
LOGGER.info("Stopping thread for task: %s", taskInfo.getTaskId());
taskInfo.getTask().stopGracefully(taskConfig);
try {
taskInfo.getResult().get(
taskConfig.getGracefulShutdownTimeout().toStandardDuration().getMillis(),
TimeUnit.MILLISECONDS
);
}
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());
if (taskInfo.shutdownFuture != null) {
taskInfo.shutdownFuture.cancel(true);
}
}
return null;
}
}
);
return taskInfo.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<ListenableFuture<Void>> shutdownFutures = new ArrayList<>();
synchronized (tasks) {
for (ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) {
shutdownFutures.add(scheduleTaskShutdown(taskWorkItem));
}
}
controlThreadExecutor.shutdown();
try {
ListenableFuture<List<Void>> shutdownFuture = Futures.successfulAsList(shutdownFutures);
shutdownFuture.get();
}
catch (Exception e) {
LOGGER.error(e, "Encountered exception when stopping all tasks.");
}
final DateTime start = DateTimes.nowUtc();
final long gracefulShutdownMillis = taskConfig.getGracefulShutdownTimeout().toStandardDuration().getMillis();
LOGGER.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) {
LOGGER.info("Finished stopping in %,dms.", elapsed);
} else {
final Set<String> stillRunning;
synchronized (tasks) {
stillRunning = ImmutableSet.copyOf(tasks.keySet());
}
LOGGER.makeAlert("Failed to stop task threads")
.addData("stillRunning", stillRunning)
.addData("elapsed", elapsed)
.emit();
LOGGER.warn(
"Executor failed to stop after %,dms, not waiting for it! Tasks still running: [%s]",
elapsed,
Joiner.on("; ").join(stillRunning)
);
}
}
catch (InterruptedException e) {
LOGGER.warn(e, "Interrupted while waiting for executor to finish.");
Thread.currentThread().interrupt();
}
} else {
LOGGER.warn("Ran out of time, not waiting for executor to finish!");
}
}
@Override
public Collection<TaskRunnerWorkItem> getRunningTasks()
{
return getTasks(RunnerTaskState.RUNNING);
}
@Override
public Collection<TaskRunnerWorkItem> getPendingTasks()
{
return getTasks(RunnerTaskState.PENDING);
}
@Nullable
@Override
public RunnerTaskState getRunnerTaskState(String taskId)
{
final ThreadingTaskRunnerWorkItem workItem = tasks.get(taskId);
return workItem == null ? null : workItem.getState();
}
private Collection<TaskRunnerWorkItem> getTasks(RunnerTaskState state)
{
synchronized (tasks) {
final List<TaskRunnerWorkItem> ret = new ArrayList<>();
for (final ThreadingTaskRunnerWorkItem taskWorkItem : tasks.values()) {
if (taskWorkItem.getState() == state) {
ret.add(taskWorkItem);
}
}
return ret;
}
}
@Override
public Optional<ScalingStats> getScalingStats()
{
return Optional.absent();
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(
Query<T> query,
Iterable<Interval> intervals
)
{
return appenderatorsManager.getQueryRunnerForIntervals(query, intervals);
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(
Query<T> query,
Iterable<SegmentDescriptor> specs
)
{
return appenderatorsManager.getQueryRunnerForSegments(query, specs);
}
protected static class ThreadingTaskRunnerWorkItem extends TaskRunnerWorkItem
{
private final Task task;
private volatile boolean shutdown = false;
private volatile ListenableFuture shutdownFuture;
private volatile RunnerTaskState state;
private ThreadingTaskRunnerWorkItem(
Task task,
ListenableFuture<TaskStatus> statusFuture
)
{
super(task.getId(), statusFuture);
this.task = task;
this.state = RunnerTaskState.PENDING;
}
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 RunnerTaskState getState()
{
return state;
}
public void setState(RunnerTaskState state)
{
this.state = state;
}
}
}

View File

@ -41,7 +41,6 @@ import org.apache.druid.concurrent.LifecycleLock;
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,7 +437,10 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
private void startWorkersHandling() throws InterruptedException
{
final CountDownLatch workerViewInitialized = new CountDownLatch(1);
DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER);
DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForService(
WorkerNodeService.DISCOVERY_SERVICE_KEY
);
druidNodeDiscovery.registerListener(
new DruidNodeDiscovery.Listener()
{

View File

@ -51,7 +51,7 @@ import org.apache.druid.query.QueryRunner;
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;
@ -77,6 +77,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
protected final AuthorizerMapper authorizerMapper;
protected final RowIngestionMetersFactory rowIngestionMetersFactory;
protected final CircularBuffer<Throwable> savedParseExceptions;
protected final AppenderatorsManager appenderatorsManager;
protected final LockGranularity lockGranularityToUse;
// Lazily initialized, to avoid calling it on the overlord when tasks are instantiated.
@ -94,7 +95,8 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
@Nullable final ChatHandlerProvider chatHandlerProvider,
final AuthorizerMapper authorizerMapper,
final RowIngestionMetersFactory rowIngestionMetersFactory,
@Nullable final String groupId
@Nullable final String groupId,
AppenderatorsManager appenderatorsManager
)
{
super(
@ -117,6 +119,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
this.authorizerMapper = authorizerMapper;
this.rowIngestionMetersFactory = rowIngestionMetersFactory;
this.runnerSupplier = Suppliers.memoize(this::createTaskRunner);
this.appenderatorsManager = appenderatorsManager;
this.lockGranularityToUse = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK)
? LockGranularity.TIME_CHUNK
: LockGranularity.SEGMENT;
@ -185,6 +188,8 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
{
if (taskConfig.isRestoreTasksOnRestart()) {
getRunner().stopGracefully();
} else {
getRunner().stopForcefully();
}
}
@ -201,7 +206,8 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
public Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox toolbox)
{
return Appenderators.createRealtime(
return appenderatorsManager.createRealtimeAppenderatorForTask(
getId(),
dataSchema,
tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()),
metrics,

View File

@ -75,6 +75,7 @@ import org.apache.druid.segment.realtime.FireDepartment;
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;
@ -200,6 +201,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
private final CircularBuffer<Throwable> savedParseExceptions;
private final String stream;
private final RowIngestionMeters rowIngestionMeters;
private final AppenderatorsManager appenderatorsManager;
private final Set<String> publishingSequences = Sets.newConcurrentHashSet();
private final List<ListenableFuture<SegmentsAndMetadata>> publishWaitList = new ArrayList<>();
@ -228,6 +230,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
final Optional<ChatHandlerProvider> chatHandlerProvider,
final CircularBuffer<Throwable> savedParseExceptions,
final RowIngestionMetersFactory rowIngestionMetersFactory,
final AppenderatorsManager appenderatorsManager,
final LockGranularity lockGranularityToUse
)
{
@ -241,6 +244,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
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;
@ -257,7 +261,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
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
@ -385,9 +389,11 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
Throwable caughtExceptionOuter = null;
try (final RecordSupplier<PartitionIdType, SequenceOffsetType> 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);
@ -874,8 +880,10 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
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) {
@ -886,7 +894,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
}
}
toolbox.getTaskReportFileWriter().write(getTaskCompletionReports(null));
toolbox.getTaskReportFileWriter().write(task.getId(), getTaskCompletionReports(null));
return TaskStatus.success(task.getId());
}
@ -1363,6 +1371,12 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
return rowIngestionMeters;
}
public void stopForcefully()
{
log.info("Stopping forcefully (status: [%s])", status);
stopRequested.set(true);
runThread.interrupt();
}
public void stopGracefully()
{

View File

@ -158,6 +158,8 @@ public abstract class WorkerTaskManager
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 abstract class WorkerTaskManager
changeHistory.addChangeRequest(new WorkerHistoryItem.TaskUpdate(latest));
taskAnnouncementChanged(latest);
log.info(
"Job's finished. Completed [%s] with status [%s]",
task.getId(),

View File

@ -26,7 +26,7 @@ import org.apache.druid.client.cache.CacheConfig;
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 class TaskToolboxTest
null,
null,
null,
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
);
}

View File

@ -28,6 +28,7 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
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.IndexMergerV9;
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 class TestUtils
.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())
);

View File

@ -51,8 +51,8 @@ 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.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.RealtimeIOConfig;
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 class AppenderatorDriverRealtimeIndexTaskTest
private File baseDir;
private File reportsFile;
private RowIngestionMetersFactory rowIngestionMetersFactory;
private AppenderatorsManager appenderatorsManager;
@Before
public void setUp() throws IOException
@ -289,6 +291,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
derbyConnector.createSegmentTable();
derbyConnector.createPendingSegmentsTable();
appenderatorsManager = new TestAppenderatorsManager();
baseDir = tempFolder.newFolder();
reportsFile = File.createTempFile("KafkaIndexTaskTestReports-" + System.currentTimeMillis(), "json");
makeToolboxFactory(baseDir);
@ -1431,7 +1435,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
null,
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
)
{
@Override
@ -1620,7 +1625,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
EasyMock.createNiceMock(DruidNode.class),
new LookupNodeService("tier"),
new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0),
new TaskReportFileWriter(reportsFile)
new SingleFileTaskReportFileWriter(reportsFile)
);
}

View File

@ -53,6 +53,7 @@ import org.apache.druid.segment.loading.SegmentLoader;
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.NumberedOverwriteShardSpec;
@ -126,6 +127,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
private final SegmentLoaderFactory segmentLoaderFactory;
private final LockGranularity lockGranularity;
private ExecutorService exec;
private AppenderatorsManager appenderatorsManager;
public CompactionTaskRunTest(LockGranularity lockGranularity)
{
@ -140,6 +142,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
}
};
segmentLoaderFactory = new SegmentLoaderFactory(getIndexIO(), getObjectMapper());
appenderatorsManager = new TestAppenderatorsManager();
this.lockGranularity = lockGranularity;
}
@ -168,7 +171,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask compactionTask = builder
@ -211,7 +215,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask compactionTask1 = builder
@ -286,7 +291,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask compactionTask = builder
@ -326,7 +332,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final Future<Pair<TaskStatus, List<DataSegment>>> compactionFuture = exec.submit(
@ -381,7 +388,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
// day segmentGranularity
@ -433,7 +441,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask compactionTask = builder
@ -479,7 +488,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask compactionTask = builder
@ -536,7 +546,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask compactionTask = builder
@ -643,7 +654,8 @@ public class CompactionTaskRunTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
return runTask(indexTask, readyLatchToCountDown, latchToAwaitBeforeRun);
@ -709,7 +721,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
null,
null,
null,
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
);
task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK);

View File

@ -96,6 +96,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
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<DataSegment, File> 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 @@ public class CompactionTaskTest
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);
}
)
)
@ -328,7 +331,8 @@ public class CompactionTaskTest
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask task = builder
.interval(COMPACTION_INTERVAL)
@ -352,7 +356,8 @@ public class CompactionTaskTest
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask task = builder
.segments(SEGMENTS)
@ -376,7 +381,8 @@ public class CompactionTaskTest
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask task = builder
@ -824,7 +830,8 @@ public class CompactionTaskTest
rowIngestionMetersFactory,
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
retryPolicyFactory,
appenderatorsManager
);
final CompactionTask task = builder
@ -1177,7 +1184,7 @@ public class CompactionTaskTest
null,
null,
null,
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
);
this.segmentFileMap = segmentFileMap;
}

View File

@ -70,6 +70,7 @@ import org.apache.druid.segment.loading.SegmentLoader;
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.firehose.LocalFirehoseFactory;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
import org.apache.druid.segment.transform.ExpressionTransform;
@ -140,6 +141,7 @@ public class IndexTaskTest extends IngestionTestBase
private static final IndexSpec indexSpec = new IndexSpec();
private final ObjectMapper jsonMapper;
private AppenderatorsManager appenderatorsManager;
private final IndexIO indexIO;
private final RowIngestionMetersFactory rowIngestionMetersFactory;
private final LockGranularity lockGranularity;
@ -157,6 +159,8 @@ public class IndexTaskTest extends IngestionTestBase
@Before
public void setup() throws IOException
{
appenderatorsManager = new TestAppenderatorsManager();
final File cacheDir = temporaryFolder.newFolder();
segmentLoader = new SegmentLoaderLocalCacheManager(
indexIO,
@ -202,7 +206,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -255,7 +260,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
Assert.assertEquals(indexTask.getId(), indexTask.getGroupId());
@ -300,7 +306,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -338,7 +345,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -372,7 +380,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -411,7 +420,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -486,7 +496,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
Assert.assertEquals("index_append_test", indexTask.getGroupId());
@ -537,7 +548,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -601,7 +613,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -654,7 +667,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -702,7 +716,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -748,7 +763,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -793,7 +809,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -872,7 +889,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -926,7 +944,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
TaskStatus status = runTask(indexTask).lhs;
@ -1022,7 +1041,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
TaskStatus status = runTask(indexTask).lhs;
@ -1148,7 +1168,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
TaskStatus status = runTask(indexTask).lhs;
@ -1265,7 +1286,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
TaskStatus status = runTask(indexTask).lhs;
@ -1365,7 +1387,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -1436,7 +1459,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
TaskStatus status = runTask(indexTask).lhs;
@ -1484,7 +1508,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;
@ -1551,7 +1576,8 @@ public class IndexTaskTest extends IngestionTestBase
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
appenderatorsManager
);
final List<DataSegment> segments = runTask(indexTask).rhs;

View File

@ -24,7 +24,7 @@ import com.google.common.base.Optional;
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 abstract class IngestionTestBase
null,
null,
null,
new TaskReportFileWriter(taskReportsFile)
new SingleFileTaskReportFileWriter(taskReportsFile)
);
if (task.isReady(box.getTaskActionClient())) {

View File

@ -19,20 +19,22 @@
package org.apache.druid.indexing.common.task;
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 TaskReportFileWriter
public class NoopTestTaskReportFileWriter implements TaskReportFileWriter
{
public NoopTestTaskFileWriter()
@Override
public void write(String id, Map<String, TaskReport> reports)
{
super(null);
}
@Override
public void write(Map<String, TaskReport> reports)
public void setObjectMapper(ObjectMapper objectMapper)
{
}
}

View File

@ -999,7 +999,7 @@ public class RealtimeIndexTaskTest
EasyMock.createNiceMock(DruidNode.class),
new LookupNodeService("tier"),
new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0),
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
);
return toolboxFactory.build(task);

View File

@ -218,7 +218,8 @@ public class TaskSerdeTest
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
null
);
final String json = jsonMapper.writeValueAsString(task);
@ -301,7 +302,8 @@ public class TaskSerdeTest
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory
rowIngestionMetersFactory,
null
);
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {

View File

@ -0,0 +1,147 @@
/*
* 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;
@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)
{
// nothing to remove
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(
Query<T> query,
Iterable<Interval> intervals
)
{
if (realtimeAppenderator != null) {
return realtimeAppenderator.getQueryRunnerForIntervals(query, intervals);
} else {
return null;
}
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(
Query<T> query,
Iterable<SegmentDescriptor> specs
)
{
if (realtimeAppenderator != null) {
return realtimeAppenderator.getQueryRunnerForSegments(query, specs);
} else {
return null;
}
}
@Override
public boolean shouldTaskMakeNodeAnnouncements()
{
return true;
}
}

View File

@ -43,9 +43,10 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
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;
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;
@ -251,7 +252,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
null,
null,
null,
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
);
}
@ -280,7 +281,8 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
return new AllowAllAuthorizer();
}
},
new DropwizardRowIngestionMetersFactory()
new DropwizardRowIngestionMetersFactory(),
new TestAppenderatorsManager()
);
}
}

View File

@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.TaskToolbox;
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 @@ public class ParallelIndexSupervisorTaskKillTest extends AbstractParallelIndexSu
ingestionSchema,
context,
indexingServiceClient,
taskClientFactory
taskClientFactory,
new TestAppenderatorsManager()
);
}

View File

@ -35,6 +35,7 @@ import org.apache.druid.indexing.common.TaskToolbox;
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;
@ -643,7 +644,8 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd
ingestionSchema,
context,
null,
taskClientFactory
taskClientFactory,
new TestAppenderatorsManager()
);
this.taskClientFactory = taskClientFactory;
}

View File

@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.TestUtils;
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;
@ -159,7 +160,8 @@ public class ParallelIndexSupervisorTaskSerdeTest
new NoopIndexingServiceClient(),
new NoopChatHandlerProvider(),
new AuthorizerMapper(Collections.emptyMap()),
new DropwizardRowIngestionMetersFactory()
new DropwizardRowIngestionMetersFactory(),
new TestAppenderatorsManager()
);
}
}

View File

@ -30,6 +30,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.Tasks;
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;
@ -144,7 +145,8 @@ public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSuperv
spec.getIngestionSpec(),
spec.getContext(),
indexingServiceClient,
null
null,
new TestAppenderatorsManager()
);
final TaskActionClient subTaskActionClient = createActionClient(subTask);
prepareTaskForLocking(subTask);
@ -532,7 +534,8 @@ public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSuperv
getIngestionSpec(),
getContext(),
null,
new LocalParallelIndexTaskClientFactory(supervisorTask)
new LocalParallelIndexTaskClientFactory(supervisorTask),
new TestAppenderatorsManager()
);
}
}

View File

@ -23,7 +23,7 @@ import com.google.common.util.concurrent.ListenableFuture;
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;
@ -105,7 +105,7 @@ public class SingleTaskBackgroundRunnerTest
node,
null,
null,
new TaskReportFileWriter(new File("fake"))
new SingleFileTaskReportFileWriter(new File("fake"))
);
runner = new SingleTaskBackgroundRunner(
toolboxFactory,

View File

@ -66,10 +66,11 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
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;
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;
@ -110,6 +111,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig;
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;
@ -228,6 +230,7 @@ public class TaskLifecycleTest
private TaskQueueConfig tqc;
private TaskConfig taskConfig;
private DataSegmentPusher dataSegmentPusher;
private AppenderatorsManager appenderatorsManager;
private int pushedSegments;
private int announcedSinks;
@ -529,6 +532,8 @@ public class TaskLifecycleTest
Preconditions.checkNotNull(taskStorage);
Preconditions.checkNotNull(emitter);
appenderatorsManager = new TestAppenderatorsManager();
taskLockbox = new TaskLockbox(taskStorage, mdc);
tac = new LocalTaskActionClientFactory(
taskStorage,
@ -552,6 +557,7 @@ public class TaskLifecycleTest
return new ArrayList<>();
}
};
return new TaskToolboxFactory(
taskConfig,
tac,
@ -622,7 +628,7 @@ public class TaskLifecycleTest
EasyMock.createNiceMock(DruidNode.class),
new LookupNodeService("tier"),
new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0),
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
);
}
@ -708,7 +714,8 @@ public class TaskLifecycleTest
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
ROW_INGESTION_METERS_FACTORY
ROW_INGESTION_METERS_FACTORY,
appenderatorsManager
);
final Optional<TaskStatus> preRunTaskStatus = tsqa.getStatus(indexTask.getId());
@ -790,7 +797,8 @@ public class TaskLifecycleTest
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
ROW_INGESTION_METERS_FACTORY
ROW_INGESTION_METERS_FACTORY,
null
);
final TaskStatus status = runTask(indexTask);
@ -1185,7 +1193,8 @@ public class TaskLifecycleTest
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
ROW_INGESTION_METERS_FACTORY
ROW_INGESTION_METERS_FACTORY,
appenderatorsManager
);
final long startTime = System.currentTimeMillis();

View File

@ -87,7 +87,7 @@ public class HttpRemoteTaskRunnerTest
{
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class);
EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER))
EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY))
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
@ -178,7 +178,7 @@ public class HttpRemoteTaskRunnerTest
{
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class);
EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER))
EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY))
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
@ -268,7 +268,7 @@ public class HttpRemoteTaskRunnerTest
{
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class);
EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER))
EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY))
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
@ -421,7 +421,7 @@ public class HttpRemoteTaskRunnerTest
{
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class);
EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER))
EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY))
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
@ -597,7 +597,7 @@ public class HttpRemoteTaskRunnerTest
{
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class);
EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER))
EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY))
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
@ -771,7 +771,7 @@ public class HttpRemoteTaskRunnerTest
{
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class);
EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER))
EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY))
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);
@ -1206,7 +1206,7 @@ public class HttpRemoteTaskRunnerTest
{
TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery();
DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class);
EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER))
EasyMock.expect(druidNodeDiscoveryProvider.getForService(WorkerNodeService.DISCOVERY_SERVICE_KEY))
.andReturn(druidNodeDiscovery);
EasyMock.replay(druidNodeDiscoveryProvider);

View File

@ -706,7 +706,8 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
chatHandlerProvider,
authorizerMapper,
rowIngestionMetersFactory,
groupId
groupId,
null
);
}

View File

@ -34,7 +34,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.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 class WorkerTaskManagerTest
null,
null,
null,
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
),
taskConfig,
location

View File

@ -39,7 +39,7 @@ import org.apache.druid.indexing.common.TestUtils;
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 @@ public class WorkerTaskMonitorTest
null,
null,
null,
new NoopTestTaskFileWriter()
new NoopTestTaskReportFileWriter()
),
taskConfig,
new NoopServiceEmitter(),

View File

@ -40,9 +40,9 @@ import java.util.concurrent.ConcurrentMap;
public abstract class DruidNodeDiscoveryProvider
{
private static final Map<String, Set<NodeType>> 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.MIDDLE_MANAGER, NodeType.INDEXER)
);
private final ConcurrentHashMap<String, ServiceDruidNodeDiscovery> serviceDiscoveryMap =

View File

@ -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;

View File

@ -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 class LookupModule implements DruidModule
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
);
}

View File

@ -0,0 +1,104 @@
/*
* 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.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;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordination.DataSegmentServerAnnouncer;
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 CliIndexerDataSegmentServerAnnouncerLifecycleHandler
{
private static final EmittingLogger LOG = new EmittingLogger(CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class);
private final DataSegmentServerAnnouncer dataSegmentServerAnnouncer;
private final LifecycleLock lifecycleLock = new LifecycleLock();
@Inject
public CliIndexerDataSegmentServerAnnouncerLifecycleHandler(
DataSegmentServerAnnouncer dataSegmentServerAnnouncer
)
{
this.dataSegmentServerAnnouncer = dataSegmentServerAnnouncer;
}
@LifecycleStart
public void start() throws IOException
{
if (!lifecycleLock.canStart()) {
throw new RuntimeException("Lifecycle lock could not start");
}
try {
if (lifecycleLock.isStarted()) {
return;
}
LOG.info("Starting...");
try {
dataSegmentServerAnnouncer.announce();
}
catch (Exception e) {
Throwables.propagateIfPossible(e, IOException.class);
throw new RuntimeException(e);
}
LOG.info("Started.");
lifecycleLock.started();
}
finally {
lifecycleLock.exitStart();
}
}
@LifecycleStop
public void stop()
{
if (!lifecycleLock.canStop()) {
throw new RuntimeException("Lifecycle lock could not stop");
}
if (!lifecycleLock.isStarted()) {
return;
}
LOG.info("Stopping...");
try {
dataSegmentServerAnnouncer.unannounce();
}
catch (Exception e) {
throw new RuntimeException(e);
}
LOG.info("Stopped.");
}
}

View File

@ -128,9 +128,7 @@ public class AppenderatorImpl implements Appenderator
*/
private final ConcurrentMap<SegmentIdWithShardSpec, Sink> sinks = new ConcurrentHashMap<>();
private final Set<SegmentIdWithShardSpec> droppingSinks = Sets.newConcurrentHashSet();
private final VersionedIntervalTimeline<String, Sink> sinkTimeline = new VersionedIntervalTimeline<>(
String.CASE_INSENSITIVE_ORDER
);
private final VersionedIntervalTimeline<String, Sink> sinkTimeline;
private final long maxBytesTuningConfig;
private final QuerySegmentWalker texasRanger;
@ -172,6 +170,55 @@ public class AppenderatorImpl implements Appenderator
CacheConfig cacheConfig,
CachePopulatorStats cachePopulatorStats
)
{
this(
schema,
tuningConfig,
metrics,
dataSegmentPusher,
objectMapper,
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
);
log.info("Created Appenderator for dataSource[%s].", schema.getDataSource());
}
/**
* 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.
*/
AppenderatorImpl(
DataSchema schema,
AppenderatorConfig tuningConfig,
FireDepartmentMetrics metrics,
DataSegmentPusher dataSegmentPusher,
ObjectMapper objectMapper,
DataSegmentAnnouncer segmentAnnouncer,
SinkQuerySegmentWalker sinkQuerySegmentWalker,
IndexIO indexIO,
IndexMerger indexMerger,
Cache cache
)
{
this.schema = Preconditions.checkNotNull(schema, "schema");
this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig");
@ -182,21 +229,21 @@ public class AppenderatorImpl implements Appenderator
this.indexIO = Preconditions.checkNotNull(indexIO, "indexIO");
this.indexMerger = Preconditions.checkNotNull(indexMerger, "indexMerger");
this.cache = cache;
this.texasRanger = conglomerate == null ? null : new SinkQuerySegmentWalker(
schema.getDataSource(),
sinkTimeline,
objectMapper,
emitter,
conglomerate,
queryExecutorService,
Preconditions.checkNotNull(cache, "cache"),
cacheConfig,
cachePopulatorStats
);
this.texasRanger = sinkQuerySegmentWalker;
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());
}
@Override
public String getDataSource()
{

View File

@ -0,0 +1,130 @@
/*
* 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 {@link Appenderator} instances.
*
* 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 {@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.
*
* 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
{
/**
* 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.
*/
<T> QueryRunner<T> getQueryRunnerForIntervals(
Query<T> query,
Iterable<Interval> intervals
);
/**
* Returns a query runner for the given segment specs over the Appenderators managed by this AppenderatorsManager.
*/
<T> QueryRunner<T> getQueryRunnerForSegments(
Query<T> query,
Iterable<SegmentDescriptor> 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();
}

View File

@ -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 <T> QueryRunner<T> getQueryRunnerForIntervals(
Query<T> query,
Iterable<Interval> intervals
)
{
throw new UOE(ERROR_MSG);
}
@Override
public <T> QueryRunner<T> getQueryRunnerForSegments(
Query<T> query,
Iterable<SegmentDescriptor> specs
)
{
throw new UOE(ERROR_MSG);
}
@Override
public boolean shouldTaskMakeNodeAnnouncements()
{
throw new UOE(ERROR_MSG);
}
}

View File

@ -0,0 +1,170 @@
/*
* 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;
/**
* 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;
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
)
{
// 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 {
batchAppenderator = Appenderators.createOffline(
schema,
config,
metrics,
dataSegmentPusher,
objectMapper,
indexIO,
indexMerger
);
return batchAppenderator;
}
}
@Override
public void removeAppenderatorForTask(String taskId)
{
// the peon only runs one task, and the process will shutdown later, don't need to do anything
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(
Query<T> query,
Iterable<Interval> 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 <T> QueryRunner<T> getQueryRunnerForSegments(
Query<T> query,
Iterable<SegmentDescriptor> 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;
}
}

View File

@ -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<String, Sink> sinkTimeline;
private final ObjectMapper objectMapper;
private final ServiceEmitter emitter;
@ -312,6 +313,11 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
);
}
public VersionedIntervalTimeline<String, Sink> getSinkTimeline()
{
return sinkTimeline;
}
public static String makeHydrantCacheIdentifier(FireHydrant input)
{
return input.getSegmentId() + "_" + input.getCount();

View File

@ -0,0 +1,199 @@
/*
* 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.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 final ConcurrentHashMap<String, SinkQuerySegmentWalker> datasourceSegmentWalkers = 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<String, Sink> 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,
indexIO,
indexMerger,
cache
);
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
);
return appenderator;
}
@Override
public void removeAppenderatorForTask(String taskId)
{
// nothing to remove presently
}
@Override
public <T> QueryRunner<T> getQueryRunnerForIntervals(
Query<T> query,
Iterable<Interval> 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 <T> QueryRunner<T> getQueryRunnerForSegments(
Query<T> query,
Iterable<SegmentDescriptor> 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;
}
}

View File

@ -32,7 +32,7 @@ import javax.ws.rs.core.Context;
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 class ChatHandlerResource
this.taskId = taskIdHolder.getTaskId();
}
@Path("/chat/{id}")
@Path("/{id}")
public Object doTaskChat(@PathParam("id") String handlerId, @Context HttpHeaders headers)
{
if (taskId != null) {

View File

@ -74,6 +74,11 @@ public class ClusterResource
entityBuilder.put(NodeType.MIDDLE_MANAGER, mmNodes);
}
Collection<Object> indexerNodes = getNodes(NodeType.INDEXER, full);
if (!indexerNodes.isEmpty()) {
entityBuilder.put(NodeType.INDEXER, indexerNodes);
}
Collection<Object> routerNodes = getNodes(NodeType.ROUTER, full);
if (!routerNodes.isEmpty()) {
entityBuilder.put(NodeType.ROUTER, routerNodes);

View File

@ -34,12 +34,50 @@ import java.util.zip.Deflater;
*/
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,
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 +245,9 @@ public class ServerConfig
", compressionLevel=" + compressionLevel +
'}';
}
public static int getDefaultNumThreads()
{
return Math.max(10, (JvmUtils.getRuntimeInfo().getAvailableProcessors() * 17) / 16 + 2) + 30;
}
}

View File

@ -62,7 +62,11 @@ public class ChatHandlerServerModule implements Module
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);

View File

@ -0,0 +1,163 @@
/*
* 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.query.lookup.LookupModule;
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)
);
}
/**
* 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) + LookupModule.LOOKUP_LISTENER_QOS_MAX_REQUESTS,
oldConfig.getQueueSize(),
oldConfig.isEnableRequestLimit(),
oldConfig.getMaxIdleTime(),
oldConfig.getDefaultQueryTimeout(),
oldConfig.getMaxScatterGatherBytes(),
oldConfig.getMaxQueryTimeout(),
oldConfig.getMaxRequestHeaderSize(),
oldConfig.getGracefulShutdownTimeout(),
oldConfig.getUnannouncePropagationDelay(),
oldConfig.getInflateBufferSize(),
oldConfig.getCompressionLevel()
);
}
}

View File

@ -40,7 +40,7 @@ public class 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 class JettyBindings
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<? extends Handler> handlerClass)
@ -60,12 +71,12 @@ public class JettyBindings
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 class JettyBindings
@Override
public String getPath()
{
return path;
return null;
}
@Override
public String[] getPaths()
{
return paths;
}
@Override

View File

@ -27,9 +27,11 @@ import org.eclipse.jetty.server.Handler;
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;
import java.util.Arrays;
import java.util.Set;
public class JettyServerInitUtils
@ -63,14 +65,23 @@ public class JettyServerInitUtils
} 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) {
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);
}
}

View File

@ -38,7 +38,6 @@ import java.util.Map;
@ExtensionPoint
public interface ServletFilterHolder
{
/**
* Get the Filter object that should be added to the servlet.
*
@ -68,12 +67,25 @@ public interface ServletFilterHolder
Map<String, String> 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()};
}
/**
* The dispatcher type that this Filter should apply to
*

View File

@ -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());
}
}

View File

@ -165,6 +165,8 @@ public class LocalDataSegmentPusherTest
{
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()

View File

@ -0,0 +1,184 @@
/*
* 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.Module;
import com.google.inject.Provides;
import com.google.inject.name.Names;
import io.airlift.airline.Command;
import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.discovery.NodeType;
import org.apache.druid.discovery.WorkerNodeService;
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.NodeTypeConfig;
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.MultipleFileTaskReportFileWriter;
import org.apache.druid.indexing.common.TaskReportFileWriter;
import org.apache.druid.indexing.overlord.TaskRunner;
import org.apache.druid.indexing.overlord.ThreadingTaskRunner;
import org.apache.druid.indexing.worker.Worker;
import org.apache.druid.indexing.worker.config.WorkerConfig;
import org.apache.druid.indexing.worker.http.ShuffleResource;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.QuerySegmentWalker;
import org.apache.druid.query.lookup.LookupModule;
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.server.DruidNode;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.http.SegmentListerResource;
import org.apache.druid.server.initialization.jetty.CliIndexerServerModule;
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. The Indexer is a task execution process that runs each task in a separate thread."
)
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<? extends Module> 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(8091);
IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder);
JsonConfigProvider.bind(binder, "druid.worker", WorkerConfig.class);
CliPeon.bindTaskConfigAndClients(binder);
binder.bind(TaskReportFileWriter.class).toInstance(new MultipleFileTaskReportFileWriter());
binder.bind(TaskRunner.class).to(ThreadingTaskRunner.class);
binder.bind(QuerySegmentWalker.class).to(ThreadingTaskRunner.class);
binder.bind(ThreadingTaskRunner.class).in(LazySingleton.class);
CliPeon.bindRowIngestionMeters(binder);
CliPeon.bindChatHandler(binder);
CliPeon.bindPeonDataSegmentHandlers(binder);
CliPeon.bindRealtimeCache(binder);
CliPeon.bindCoordinatorHandoffNotiferAndClient(binder);
CliMiddleManager.bindWorkerManagementClasses(binder);
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);
LifecycleModule.register(binder, CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class);
Jerseys.addResource(binder, ShuffleResource.class);
LifecycleModule.register(binder, Server.class, RemoteChatHandler.class);
bindAnnouncer(
binder,
DiscoverySideEffectsProvider.builder(NodeType.INDEXER)
.serviceClasses(
ImmutableList.of(LookupNodeService.class, WorkerNodeService.class)
)
.build()
);
}
@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 CliIndexerServerModule(properties),
new LookupModule()
);
}
}

View File

@ -59,6 +59,8 @@ 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.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;
@ -121,16 +123,16 @@ public class CliMiddleManager extends ServerRunnable
.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)
.in(LazySingleton.class);
Jerseys.addResource(binder, ShuffleResource.class);
LifecycleModule.register(binder, Server.class);
@ -172,4 +174,13 @@ public class CliMiddleManager extends ServerRunnable
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);
}
}

View File

@ -95,6 +95,8 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorResource;
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 class CliOverlord extends ServerRunnable
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);
}

View File

@ -58,6 +58,7 @@ import org.apache.druid.guice.annotations.Json;
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.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.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;
@ -166,62 +169,13 @@ public class CliPeon extends GuiceRunnable
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<String, RowIngestionMetersFactory> rowIngestionMetersHandlerProviderBinder =
PolyBind.optionBinder(binder, Key.get(RowIngestionMetersFactory.class));
rowIngestionMetersHandlerProviderBinder
.addBinding("dropwizard")
.to(DropwizardRowIngestionMetersFactory.class)
.in(LazySingleton.class);
binder.bind(DropwizardRowIngestionMetersFactory.class).in(LazySingleton.class);
bindRowIngestionMeters(binder);
PolyBind.createChoice(
binder,
"druid.indexer.task.chathandler.type",
Key.get(ChatHandlerProvider.class),
Key.get(ServiceAnnouncingChatHandlerProvider.class)
);
final MapBinder<String, ChatHandlerProvider> 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);
bindChatHandler(binder);
binder.bind(NoopChatHandlerProvider.class).in(LazySingleton.class);
bindTaskConfigAndClients(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<IndexTaskClientFactory<ParallelIndexTaskClient>>(){})
.to(ParallelIndexTaskClientFactory.class)
.in(LazySingleton.class);
binder.bind(RetryPolicyFactory.class).in(LazySingleton.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);
bindPeonDataSegmentHandlers(binder);
binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class);
LifecycleModule.register(binder, ExecutorLifecycle.class);
@ -232,7 +186,7 @@ public class CliPeon extends GuiceRunnable
);
binder.bind(TaskReportFileWriter.class).toInstance(
new TaskReportFileWriter(
new SingleFileTaskReportFileWriter(
new File(taskReportPath)
)
);
@ -241,53 +195,20 @@ public class CliPeon extends GuiceRunnable
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)
bindCoordinatorHandoffNotiferAndClient(binder);
binder.bind(AppenderatorsManager.class)
.to(PeonAppenderatorsManager.class)
.in(LazySingleton.class);
binder.bind(CoordinatorClient.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)));
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<String, TaskActionClientFactory> 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)
@ -383,4 +304,120 @@ public class CliPeon extends GuiceRunnable
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<String, RowIngestionMetersFactory> 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<String, ChatHandlerProvider> 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<String, TaskActionClientFactory> 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<IndexTaskClientFactory<ParallelIndexTaskClient>>(){})
.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);
}
}

View File

@ -60,6 +60,7 @@ public class Main
CliHistorical.class,
CliBroker.class,
CliOverlord.class,
CliIndexer.class,
CliMiddleManager.class,
CliRouter.class
);

View File

@ -421,6 +421,13 @@ public class SystemSchemaTest extends CalciteTestBase
DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0))
);
private final DiscoveryDruidNode indexer = new DiscoveryDruidNode(
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))
);
private final ImmutableDruidServer druidServer1 = new ImmutableDruidServer(
new DruidServerMetadata("server1", "localhost:0000", null, 5L, ServerType.REALTIME, DruidServer.DEFAULT_TIER, 0),
1L,
@ -686,6 +693,8 @@ public class SystemSchemaTest extends CalciteTestBase
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)
@ -701,6 +710,9 @@ public class SystemSchemaTest extends CalciteTestBase
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();
@ -710,6 +722,7 @@ public class SystemSchemaTest extends CalciteTestBase
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();
@ -726,7 +739,8 @@ public class SystemSchemaTest extends CalciteTestBase
routerNodeDiscovery,
historicalNodeDiscovery,
mmNodeDiscovery,
peonNodeDiscovery
peonNodeDiscovery,
indexerNodeDiscovery
);
DataContext dataContext = new DataContext()
@ -757,7 +771,7 @@ public class SystemSchemaTest extends CalciteTestBase
};
final List<Object[]> 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",
@ -782,6 +796,17 @@ public class SystemSchemaTest extends CalciteTestBase
);
verifyServerRow(
rows.get(2),
"indexerHost:8091",
"indexerHost",
8091,
-1,
"indexer",
null,
0,
0
);
verifyServerRow(
rows.get(3),
"localhost:8080",
"localhost",
8080,
@ -792,7 +817,7 @@ public class SystemSchemaTest extends CalciteTestBase
0
);
verifyServerRow(
rows.get(3),
rows.get(4),
"localhost:8081",
"localhost",
8081,
@ -803,7 +828,7 @@ public class SystemSchemaTest extends CalciteTestBase
0
);
verifyServerRow(
rows.get(4),
rows.get(5),
"localhost:8082",
"localhost",
8082,
@ -814,7 +839,7 @@ public class SystemSchemaTest extends CalciteTestBase
0
);
verifyServerRow(
rows.get(5),
rows.get(6),
"localhost:8083",
"localhost",
8083,
@ -825,7 +850,7 @@ public class SystemSchemaTest extends CalciteTestBase
1000
);
verifyServerRow(
rows.get(6),
rows.get(7),
"localhost:8090",
"localhost",
8090,
@ -836,7 +861,7 @@ public class SystemSchemaTest extends CalciteTestBase
0
);
verifyServerRow(
rows.get(7),
rows.get(8),
"localhost:8888",
"localhost",
8888,
@ -847,7 +872,7 @@ public class SystemSchemaTest extends CalciteTestBase
0
);
verifyServerRow(
rows.get(8),
rows.get(9),
"mmHost:8091",
"mmHost",
8091,
@ -858,7 +883,7 @@ public class SystemSchemaTest extends CalciteTestBase
0
);
verifyServerRow(
rows.get(9),
rows.get(10),
"peonHost:8080",
"peonHost",
8080,