From fb5a78789063c01c5ee3cadd89259dd5316cc9e9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 14 Mar 2018 18:24:03 -0700 Subject: [PATCH 01/47] Native parallel indexing without shuffle --- .../data/input/FiniteFirehoseFactory.java | 60 ++ .../io/druid/data/input/FirehoseFactory.java | 5 + .../java/io/druid/data/input/InputSplit.java | 38 ++ .../AbstractTextFilesFirehoseFactory.java | 40 +- .../java/io/druid/indexer/TaskReport.java | 55 ++ ...fetchableTextFilesFirehoseFactoryTest.java | 8 + .../StaticAzureBlobStoreFirehoseFactory.java | 18 + .../StaticCloudFilesFirehoseFactory.java | 18 + .../StaticGoogleBlobStoreFirehoseFactory.java | 18 + .../indexing/kafka/KafkaIndexTaskTest.java | 4 +- .../firehose/s3/StaticS3FirehoseFactory.java | 29 + .../CountingActionBasedSegmentAllocator.java | 77 +++ .../io/druid/indexing/common/Counters.java | 72 +++ .../io/druid/indexing/common/TaskStatus.java | 33 +- .../CountingSegmentAllocateAction.java | 202 +++++++ .../indexing/common/actions/TaskAction.java | 1 + .../common/actions/TaskActionToolbox.java | 11 +- .../druid/indexing/common/task/IndexTask.java | 4 +- .../ParallelIndexSinglePhaseIOConfig.java | 41 ++ ...ParallelIndexSinglePhaseIngestionSpec.java | 68 +++ .../task/ParallelIndexSinglePhaseSubTask.java | 544 ++++++++++++++++++ ...arallelIndexSinglePhaseSupervisorTask.java | 362 ++++++++++++ .../ParallelIndexSinglePhaseTuningConfig.java | 176 ++++++ .../io/druid/indexing/common/task/Task.java | 2 + .../indexing/common/task/TaskMonitor.java | 192 +++++++ .../druid/indexing/overlord/TaskLockbox.java | 18 +- .../druid/indexing/overlord/TaskMaster.java | 4 +- .../CountingSegmentAllocateActionTest.java | 149 +++++ .../common/actions/TaskActionTestKit.java | 4 +- ...penderatorDriverRealtimeIndexTaskTest.java | 4 +- .../common/task/IngestionTestBase.java | 107 ++++ ...lelIndexSinglePhaseSupervisorTaskTest.java | 392 +++++++++++++ .../common/task/RealtimeIndexTaskTest.java | 4 +- .../indexing/common/task/TaskMonitorTest.java | 146 +++++ .../IngestSegmentFirehoseFactoryTest.java | 11 +- .../indexing/overlord/TaskLifecycleTest.java | 13 +- .../io/druid/client/indexing/ClientQuery.java | 25 + .../indexing/IndexingServiceClient.java | 75 ++- .../io/druid/client/indexing/TaskStatus.java | 114 ++++ .../client/indexing/TaskStatusResponse.java | 86 +++ .../IndexerMetadataStorageCoordinator.java | 7 +- .../overlord/supervisor/SupervisorSpec.java | 6 + .../IndexerSQLMetadataStorageCoordinator.java | 9 - .../io/druid/segment/indexing/IOConfig.java | 2 +- .../firehose/HttpFirehoseFactory.java | 22 + .../firehose/LocalFirehoseFactory.java | 13 +- .../timeline/partition/NumberedShardSpec.java | 25 + .../src/main/java/io/druid/cli/CliPeon.java | 2 + .../java/io/druid/cli/CliRealtimeExample.java | 2 +- 49 files changed, 3267 insertions(+), 51 deletions(-) create mode 100644 api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java create mode 100644 api/src/main/java/io/druid/data/input/InputSplit.java create mode 100644 api/src/main/java/io/druid/indexer/TaskReport.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/Counters.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIngestionSpec.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java create mode 100644 server/src/main/java/io/druid/client/indexing/ClientQuery.java create mode 100644 server/src/main/java/io/druid/client/indexing/TaskStatus.java create mode 100644 server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java diff --git a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java new file mode 100644 index 000000000000..012b8914d290 --- /dev/null +++ b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java @@ -0,0 +1,60 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input; + +import io.druid.data.input.impl.InputRowParser; + +import java.io.IOException; +import java.util.Iterator; + +/** + * {@link FirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is + * limited. + * + * @param parser type + * @param input split type + */ +public interface FiniteFirehoseFactory extends FirehoseFactory +{ + /** + * Returns true if the {@link FirehoseFactory} supports parallel batch indexing. + */ + @Override + default boolean isSplittable() + { + return true; + } + + /** + * Returns an iterator of {@link InputSplit}s. + */ + Iterator> getSplits() throws IOException; + + /** + * Returns number of splits returned by {@link #getSplits()}. + */ + int getNumSplits() throws IOException; + + /** + * Returns the same {@link FiniteFirehoseFactory} but with the given {@link InputSplit}. The returned + * {@link FiniteFirehoseFactory} is used by sub tasks in parallel batch indexing. + */ + FiniteFirehoseFactory withSplit(InputSplit split); +} diff --git a/api/src/main/java/io/druid/data/input/FirehoseFactory.java b/api/src/main/java/io/druid/data/input/FirehoseFactory.java index f99369bee4b6..7558e47be9b5 100644 --- a/api/src/main/java/io/druid/data/input/FirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FirehoseFactory.java @@ -72,4 +72,9 @@ default Firehose connect(T parser, @Nullable File temporaryDirectory) throws IOE { return connect(parser); } + + default boolean isSplittable() + { + return false; + } } diff --git a/api/src/main/java/io/druid/data/input/InputSplit.java b/api/src/main/java/io/druid/data/input/InputSplit.java new file mode 100644 index 000000000000..9e775c796391 --- /dev/null +++ b/api/src/main/java/io/druid/data/input/InputSplit.java @@ -0,0 +1,38 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input; + +/** + * Input unit for distributed batch ingestion. Used in {@link FiniteFirehoseFactory}. + */ +public class InputSplit +{ + private final T split; + + public InputSplit(T split) + { + this.split = split; + } + + public T get() + { + return split; + } +} diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index 4059715a22a8..a3f124046d5f 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -22,8 +22,9 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import io.druid.data.input.FiniteFirehoseFactory; import io.druid.data.input.Firehose; -import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputSplit; import io.druid.java.util.common.logger.Logger; import org.apache.commons.io.Charsets; import org.apache.commons.io.IOUtils; @@ -36,6 +37,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.stream.Collectors; /** * This is an abstract class for firehose factory for making firehoses reading text files. @@ -44,7 +46,7 @@ * @param object type representing input data */ public abstract class AbstractTextFilesFirehoseFactory - implements FirehoseFactory + implements FiniteFirehoseFactory { private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class); @@ -53,9 +55,7 @@ public abstract class AbstractTextFilesFirehoseFactory @Override public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException { - if (objects == null) { - objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); - } + initializeObjectsIfNeeded(); final Iterator iterator = objects.iterator(); return new FileIteratingFirehose( new Iterator() @@ -90,6 +90,36 @@ public LineIterator next() ); } + protected void initializeObjectsIfNeeded() throws IOException + { + if (objects == null) { + objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects")); + } + } + + public List getObjects() + { + return objects; + } + + @Override + public Iterator> getSplits() throws IOException + { + initializeObjectsIfNeeded(); + return getObjects() + .stream() + .map(InputSplit::new) + .collect(Collectors.toList()) + .iterator(); + } + + @Override + public int getNumSplits() throws IOException + { + initializeObjectsIfNeeded(); + return getObjects().size(); + } + /** * Initialize objects to be read by this firehose. Since firehose factories are constructed whenever * io.druid.indexing.common.task.Task objects are deserialized, actual initialization of objects is deferred diff --git a/api/src/main/java/io/druid/indexer/TaskReport.java b/api/src/main/java/io/druid/indexer/TaskReport.java new file mode 100644 index 000000000000..74d64e658547 --- /dev/null +++ b/api/src/main/java/io/druid/indexer/TaskReport.java @@ -0,0 +1,55 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexer; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * TaskReport can be optionally included in io.druid.indexing.common.TaskStatus to report some ingestion results to + * Supervisors or supervisorTasks. See ParallelIndexSinglePhaseSupervisorTask and ParallelIndexSinglePhaseSubTask + * as an example. + */ +public class TaskReport +{ + private final String taskId; + private final Object payload; // can't use generic to not change TaskStatus + + @JsonCreator + public TaskReport( + @JsonProperty("taskId") String taskId, + @JsonProperty("payload") Object payload) + { + this.taskId = taskId; + this.payload = payload; + } + + @JsonProperty + public String getTaskId() + { + return taskId; + } + + @JsonProperty + public Object getPayload() + { + return payload; + } +} diff --git a/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java index fc8c5a7b519a..17dec13839bc 100644 --- a/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/api/src/test/java/io/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java @@ -24,7 +24,9 @@ import com.google.common.base.Predicate; import com.google.common.collect.Lists; import com.google.common.io.CountingOutputStream; +import io.druid.data.input.FiniteFirehoseFactory; import io.druid.data.input.Firehose; +import io.druid.data.input.InputSplit; import io.druid.data.input.Row; import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; @@ -606,6 +608,12 @@ protected InputStream openObjectStream(File object, long start) throws IOExcepti private int readCount; private int numConnectionResets; + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + throw new UnsupportedOperationException(); + } + private class TestInputStream extends InputStream { private static final int NUM_READ_COUNTS_BEFORE_ERROR = 10; diff --git a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java index e3ad8b432aa7..d88b1490090f 100644 --- a/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java +++ b/extensions-contrib/azure-extensions/src/main/java/io/druid/firehose/azure/StaticAzureBlobStoreFirehoseFactory.java @@ -24,6 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; import io.druid.storage.azure.AzureByteSource; @@ -33,6 +36,7 @@ import java.io.IOException; import java.io.InputStream; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; @@ -143,4 +147,18 @@ protected Predicate getRetryCondition() { return AzureUtils.AZURE_RETRY; } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + return new StaticAzureBlobStoreFirehoseFactory( + azureStorage, + Collections.singletonList(split.get()), + getMaxCacheCapacityBytes(), + getMaxFetchCapacityBytes(), + getPrefetchTriggerBytes(), + getFetchTimeout(), + getMaxFetchRetry() + ); + } } diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java index 5f39e7e5a446..f512f768d431 100644 --- a/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java +++ b/extensions-contrib/cloudfiles-extensions/src/main/java/io/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java @@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Predicate; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.logger.Logger; @@ -34,6 +37,7 @@ import java.io.IOException; import java.io.InputStream; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; @@ -142,4 +146,18 @@ protected Predicate getRetryCondition() { return CloudFilesUtils.CLOUDFILESRETRY; } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + return new StaticCloudFilesFirehoseFactory( + cloudFilesApi, + Collections.singletonList(split.get()), + getMaxCacheCapacityBytes(), + getMaxFetchCapacityBytes(), + getPrefetchTriggerBytes(), + getFetchTimeout(), + getMaxFetchRetry() + ); + } } diff --git a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java index 0d5d99959609..7f5c78e4a9da 100644 --- a/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java +++ b/extensions-contrib/google-extensions/src/main/java/io/druid/firehose/google/StaticGoogleBlobStoreFirehoseFactory.java @@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Predicate; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; import io.druid.storage.google.GoogleByteSource; @@ -32,6 +35,7 @@ import java.io.IOException; import java.io.InputStream; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; @@ -134,5 +138,19 @@ protected Predicate getRetryCondition() { return GoogleUtils.GOOGLE_RETRY; } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + return new StaticGoogleBlobStoreFirehoseFactory( + storage, + Collections.singletonList(split.get()), + getMaxCacheCapacityBytes(), + getMaxFetchCapacityBytes(), + getPrefetchTriggerBytes(), + getFetchTimeout(), + getMaxFetchRetry() + ); + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index dfc20525513b..ef9c59e40da1 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -38,6 +38,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import io.druid.indexing.common.Counters; import io.druid.java.util.emitter.EmittingLogger; import io.druid.java.util.emitter.core.NoopEmitter; import io.druid.java.util.emitter.service.ServiceEmitter; @@ -1958,7 +1959,8 @@ public boolean checkPointDataSourceMetadata( )); return true; } - } + }, + new Counters() ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index fa649dc495c7..d054b7969ae3 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -24,6 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.IAE; @@ -41,6 +44,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.stream.Collectors; @@ -258,4 +262,29 @@ protected Predicate getRetryCondition() { return S3Utils.S3RETRY; } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + final String authority = split.get().getBucketName(); + final String path = split.get().getKey(); + final URI splitUri; + if (authority.endsWith("/") && path.startsWith("/")) { + splitUri = URI.create(authority + path.substring(1)); + } else if (!authority.endsWith("/") && !path.startsWith("/")) { + splitUri = URI.create(authority + "/" + path); + } else { + splitUri = URI.create(authority + path); + } + return new StaticS3FirehoseFactory( + s3Client, + Collections.singletonList(splitUri), + null, + getMaxCacheCapacityBytes(), + getMaxFetchCapacityBytes(), + getPrefetchTriggerBytes(), + getFetchTimeout(), + getMaxFetchRetry() + ); + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java new file mode 100644 index 000000000000..4319d2e0cc0f --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java @@ -0,0 +1,77 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.appenderator; + +import com.google.common.base.Preconditions; +import io.druid.data.input.InputRow; +import io.druid.indexing.common.actions.CountingSegmentAllocateAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.segment.indexing.granularity.GranularitySpec; +import io.druid.segment.realtime.appenderator.SegmentAllocator; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class CountingActionBasedSegmentAllocator implements SegmentAllocator +{ + private final TaskActionClient taskActionClient; + private final String dataSource; + private final GranularitySpec granularitySpec; + private final Map> shardSpecs; + private final Map versions; + + public CountingActionBasedSegmentAllocator( + TaskActionClient taskActionClient, + String dataSource, + GranularitySpec granularitySpec, + Map> shardSpecs, + Map versions + ) + { + this.taskActionClient = Preconditions.checkNotNull(taskActionClient, "taskActionClient"); + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.granularitySpec = Preconditions.checkNotNull(granularitySpec, "granularitySpec"); + this.shardSpecs = Preconditions.checkNotNull(shardSpecs, "shardSpecs"); + this.versions = Preconditions.checkNotNull(versions, "versions"); + } + + @Override + public SegmentIdentifier allocate( + InputRow row, + String sequenceName, + String previousSegmentId, + boolean skipSegmentLineageCheck + ) throws IOException + { + return taskActionClient.submit( + new CountingSegmentAllocateAction( + dataSource, + row.getTimestamp(), + granularitySpec, + shardSpecs, + versions + ) + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/Counters.java b/indexing-service/src/main/java/io/druid/indexing/common/Counters.java new file mode 100644 index 000000000000..8b8b2ac8abb6 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/Counters.java @@ -0,0 +1,72 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common; + +import com.google.common.util.concurrent.AtomicDouble; + +import javax.annotation.Nullable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BinaryOperator; + +public class Counters +{ + private final ConcurrentMap intCounters = new ConcurrentHashMap<>(); + private final ConcurrentMap doubleCounters = new ConcurrentHashMap<>(); + private final ConcurrentMap objectCounters = new ConcurrentHashMap<>(); + + public int increment(String key, int val) + { + return intCounters.computeIfAbsent(key, k -> new AtomicInteger()).addAndGet(val); + } + + public double increment(String key, double val) + { + return doubleCounters.computeIfAbsent(key, k -> new AtomicDouble()).addAndGet(val); + } + + public Object increment(String key, Object obj, BinaryOperator mergeFunction) + { + return objectCounters.computeIfAbsent(key, k -> new AtomicReference()).accumulateAndGet(obj, mergeFunction); + } + + @Nullable + public Integer getIntCounter(String key) + { + final AtomicInteger atomicInteger = intCounters.get(key); + return atomicInteger == null ? null : atomicInteger.get(); + } + + @Nullable + public Double getDoubleCounter(String key) + { + final AtomicDouble atomicDouble = doubleCounters.get(key); + return atomicDouble == null ? null : atomicDouble.get(); + } + + @Nullable + public Object getObjectCounter(String key) + { + final AtomicReference atomicReference = objectCounters.get(key); + return atomicReference == null ? null : atomicReference.get(); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java index b0249440c7a1..792bc976326e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java @@ -24,8 +24,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; +import io.druid.indexer.TaskReport; import io.druid.indexer.TaskState; +import javax.annotation.Nullable; + /** * Represents the status of a task from the perspective of the coordinator. The task may be ongoing * ({@link #isComplete()} false) or it may be complete ({@link #isComplete()} true). @@ -36,37 +39,45 @@ public class TaskStatus { public static TaskStatus running(String taskId) { - return new TaskStatus(taskId, TaskState.RUNNING, -1); + return new TaskStatus(taskId, TaskState.RUNNING, null, -1); } public static TaskStatus success(String taskId) { - return new TaskStatus(taskId, TaskState.SUCCESS, -1); + return new TaskStatus(taskId, TaskState.SUCCESS, null, -1); + } + + public static TaskStatus success(String taskId, TaskReport report) + { + return new TaskStatus(taskId, TaskState.SUCCESS, report, -1); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, TaskState.FAILED, -1); + return new TaskStatus(taskId, TaskState.FAILED, null, -1); } public static TaskStatus fromCode(String taskId, TaskState code) { - return new TaskStatus(taskId, code, -1); + return new TaskStatus(taskId, code, null, -1); } private final String id; private final TaskState status; + private final TaskReport report; private final long duration; @JsonCreator protected TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") TaskState status, + @JsonProperty("report") @Nullable TaskReport report, @JsonProperty("duration") long duration ) { this.id = id; this.status = status; + this.report = report; this.duration = duration; // Check class invariants. @@ -86,6 +97,12 @@ public TaskState getStatusCode() return status; } + @JsonProperty("report") + public TaskReport getReport() + { + return report; + } + @JsonProperty("duration") public long getDuration() { @@ -141,7 +158,7 @@ public boolean isFailure() public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, _duration); + return new TaskStatus(id, status, report, _duration); } @Override @@ -156,13 +173,14 @@ public boolean equals(Object o) TaskStatus that = (TaskStatus) o; return duration == that.duration && java.util.Objects.equals(id, that.id) && - status == that.status; + status == that.status && + java.util.Objects.equals(report, that.report); } @Override public int hashCode() { - return java.util.Objects.hash(id, status, duration); + return java.util.Objects.hash(id, status, report, duration); } @Override @@ -171,6 +189,7 @@ public String toString() return Objects.toStringHelper(this) .add("id", id) .add("status", status) + .add("report", report) .add("duration", duration) .toString(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java new file mode 100644 index 000000000000..27cc538ca75b --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java @@ -0,0 +1,202 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.KeyDeserializer; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import io.druid.indexing.common.Counters; +import io.druid.indexing.common.task.Task; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Intervals; +import io.druid.segment.indexing.granularity.GranularitySpec; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; + +public class CountingSegmentAllocateAction implements TaskAction +{ + private final String dataSource; + private final DateTime timestamp; + private final GranularitySpec granularitySpec; + @JsonDeserialize(keyUsing = IntervalDeserializer.class) + private final Map> shardSpecs; + @JsonDeserialize(keyUsing = IntervalDeserializer.class) + private final Map versions; + + @JsonCreator + public CountingSegmentAllocateAction( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("timestamp") DateTime timestamp, + @JsonProperty("granularitySpec") GranularitySpec granularitySpec, + @JsonProperty("shardSpecs") Map> shardSpecs, + @JsonProperty("versions") Map versions + ) + { + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.timestamp = Preconditions.checkNotNull(timestamp, "timestamp"); + this.granularitySpec = Preconditions.checkNotNull(granularitySpec, "granularitySpec"); + this.shardSpecs = Preconditions.checkNotNull(shardSpecs, "shardSpecs"); + this.versions = Preconditions.checkNotNull(versions, "versions"); + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public DateTime getTimestamp() + { + return timestamp; + } + + @JsonProperty + public GranularitySpec getGranularitySpec() + { + return granularitySpec; + } + + @JsonProperty + public Map> getShardSpecs() + { + return shardSpecs; + } + + @JsonProperty + public Map getVersions() + { + return versions; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public SegmentIdentifier perform(Task task, TaskActionToolbox toolbox) + { + Optional maybeInterval = granularitySpec.bucketInterval(timestamp); + if (!maybeInterval.isPresent()) { + throw new ISE("Could not find interval for timestamp [%s]", timestamp); + } + + final Interval interval = maybeInterval.get(); + if (!shardSpecs.containsKey(interval)) { + throw new ISE("Could not find shardSpec for interval[%s]", interval); + } + + final Counters counters = toolbox.getCounters(); + + final int partitionNum = counters.increment(interval.toString(), 1); + return new SegmentIdentifier( + dataSource, + interval, + findVersion(versions, interval), + new NumberedShardSpec(partitionNum, 0) + ); + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return "CountingSegmentAllocateAction{" + + "dataSource='" + dataSource + '\'' + + ", timestamp=" + timestamp + + ", granularitySpec=" + granularitySpec + + ", shardSpecs=" + shardSpecs + + ", versions=" + versions + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final CountingSegmentAllocateAction that = (CountingSegmentAllocateAction) o; + if (!dataSource.equals(that.dataSource)) { + return false; + } + if (!timestamp.equals(that.timestamp)) { + return false; + } + if (!granularitySpec.equals(that.granularitySpec)) { + return false; + } + if (!shardSpecs.equals(that.shardSpecs)) { + return false; + } + return versions.equals(that.versions); + } + + @Override + public int hashCode() + { + return Objects.hash(dataSource, timestamp, granularitySpec, shardSpecs, versions); + } + + private static String findVersion(Map versions, Interval interval) + { + return versions.entrySet().stream() + .filter(entry -> entry.getKey().contains(interval)) + .map(Entry::getValue) + .findFirst() + .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); + } + + public static class IntervalDeserializer extends KeyDeserializer + { + @Override + public Object deserializeKey(String s, DeserializationContext deserializationContext) + { + return Intervals.of(s); + } + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index 0da85f4803bf..b60c3622d799 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -39,6 +39,7 @@ @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), @JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class), @JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class), + @JsonSubTypes.Type(name = "countingSegmentAllocate", value = CountingSegmentAllocateAction.class), @JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class), @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class) }) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java index 4d08465b99e4..2bc0b771506c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java @@ -20,6 +20,7 @@ package io.druid.indexing.common.actions; import com.google.inject.Inject; +import io.druid.indexing.common.Counters; import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.TaskLockbox; @@ -31,19 +32,22 @@ public class TaskActionToolbox private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private final ServiceEmitter emitter; private final SupervisorManager supervisorManager; + private final Counters counters; @Inject public TaskActionToolbox( TaskLockbox taskLockbox, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, ServiceEmitter emitter, - SupervisorManager supervisorManager + SupervisorManager supervisorManager, + Counters counters ) { this.taskLockbox = taskLockbox; this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; this.emitter = emitter; this.supervisorManager = supervisorManager; + this.counters = counters; } public TaskLockbox getTaskLockbox() @@ -65,4 +69,9 @@ public SupervisorManager getSupervisorManager() { return supervisorManager; } + + public Counters getCounters() + { + return counters; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 7a4dff26c85b..a2ea87ae68a2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -67,9 +67,9 @@ import io.druid.segment.realtime.RealtimeMetricsMonitor; import io.druid.segment.realtime.appenderator.Appenderator; import io.druid.segment.realtime.appenderator.AppenderatorConfig; -import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import io.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import io.druid.segment.realtime.appenderator.SegmentAllocator; import io.druid.segment.realtime.appenderator.SegmentIdentifier; @@ -563,7 +563,7 @@ private boolean generateAndPublishSegments( final TaskToolbox toolbox, final DataSchema dataSchema, final ShardSpecs shardSpecs, - Map versions, + final Map versions, final FirehoseFactory firehoseFactory, final File firehoseTempDir ) throws IOException, InterruptedException diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java new file mode 100644 index 000000000000..b2296a1fda78 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java @@ -0,0 +1,41 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonTypeName; +import io.druid.data.input.FirehoseFactory; +import io.druid.indexing.common.task.IndexTask.IndexIOConfig; + +import javax.annotation.Nullable; + +/** + * Same with {@link IndexIOConfig} except its JSON type name. + */ +@JsonTypeName("parallelIndexSinglePhase") +public class ParallelIndexSinglePhaseIOConfig extends IndexIOConfig +{ + public ParallelIndexSinglePhaseIOConfig( + FirehoseFactory firehoseFactory, + @Nullable Boolean appendToExisting + ) + { + super(firehoseFactory, appendToExisting); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIngestionSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIngestionSpec.java new file mode 100644 index 000000000000..c4f363852b37 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIngestionSpec.java @@ -0,0 +1,68 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.IngestionSpec; + +public class ParallelIndexSinglePhaseIngestionSpec + extends IngestionSpec +{ + private final DataSchema dataSchema; + private final ParallelIndexSinglePhaseIOConfig ioConfig; + private final ParallelIndexSinglePhaseTuningConfig tuningConfig; + + @JsonCreator + public ParallelIndexSinglePhaseIngestionSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("ioConfig") ParallelIndexSinglePhaseIOConfig ioConfig, + @JsonProperty("tuningConfig") ParallelIndexSinglePhaseTuningConfig tuningConfig + ) + { + super(dataSchema, ioConfig, tuningConfig); + + this.dataSchema = dataSchema; + this.ioConfig = ioConfig; + this.tuningConfig = tuningConfig == null ? ParallelIndexSinglePhaseTuningConfig.defaultConfig() : tuningConfig; + } + + @Override + @JsonProperty("dataSchema") + public DataSchema getDataSchema() + { + return dataSchema; + } + + @Override + @JsonProperty("ioConfig") + public ParallelIndexSinglePhaseIOConfig getIOConfig() + { + return ioConfig; + } + + @Override + @JsonProperty("tuningConfig") + public ParallelIndexSinglePhaseTuningConfig getTuningConfig() + { + return tuningConfig; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java new file mode 100644 index 000000000000..a09cfdfe62b3 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java @@ -0,0 +1,544 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputRow; +import io.druid.indexer.TaskReport; +import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.appenderator.CountingActionBasedSegmentAllocator; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.IndexTask.IndexIOConfig; +import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.common.task.IndexTask.ShardSpecs; +import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.JodaUtils; +import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.common.parsers.ParseException; +import io.druid.query.DruidMetrics; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.granularity.GranularitySpec; +import io.druid.segment.realtime.FireDepartment; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.RealtimeMetricsMonitor; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import io.druid.segment.realtime.appenderator.Appenderators; +import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver; +import io.druid.segment.realtime.appenderator.BatchAppenderatorDriver; +import io.druid.segment.realtime.appenderator.SegmentAllocator; +import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; +import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.timeline.partition.ShardSpec; +import org.codehaus.plexus.util.FileUtils; +import org.joda.time.Interval; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * A worker task of {@link ParallelIndexSinglePhaseSupervisorTask}. Similar to {@link IndexTask}, but this class returns + * a {@link TaskReport} to {@link ParallelIndexSinglePhaseSupervisorTask} including the information of pushed segments + * once it finishes its work. + */ +public class ParallelIndexSinglePhaseSubTask extends AbstractTask +{ + private static final Logger log = new Logger(ParallelIndexSinglePhaseSubTask.class); + private static final String TYPE = "parallelIndexSinglePhaseSubIndex"; + + private final IndexTask.IndexIngestionSpec ingestionSchema; + + @JsonCreator + public ParallelIndexSinglePhaseSubTask( + @JsonProperty("id") final String id, + @JsonProperty("groupId") final String groupId, + @JsonProperty("resource") final TaskResource taskResource, + @JsonProperty("spec") final IndexTask.IndexIngestionSpec ingestionSchema, + @JsonProperty("context") final Map context + ) + { + super( + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + groupId, + taskResource, + ingestionSchema.getDataSchema().getDataSource(), + context + ); + + this.ingestionSchema = ingestionSchema; + + if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) { + throw new UnsupportedOperationException("Guaranteed rollup is not supported"); + } + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) + { + final Optional> intervals = ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals(); + + return !intervals.isPresent() || checkLockAcquired(taskActionClient, intervals.get()); + } + + private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) + { + // TODO: check when intervals are not specified + try { + Tasks.tryAcquireExclusiveLocks(actionClient, intervals); + return true; + } + catch (Exception e) { + log.error(e, "Failed to acquire locks for intervals[%s]", intervals); + return false; + } + } + + @JsonProperty("spec") + public IndexTask.IndexIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + final boolean determineIntervals = !ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals() + .isPresent(); + + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + + if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { + // pass toolbox to Firehose + ((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox); + } + + final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + // Firehose temporary directory is automatically removed when this IndexTask completes. + FileUtils.forceMkdir(firehoseTempDir); + + final IndexTask.ShardSpecs shardSpecs = determineShardSpecs(firehoseFactory, firehoseTempDir); + + final DataSchema dataSchema; + final Map versions; + if (determineIntervals) { + final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); + intervals.addAll(shardSpecs.getIntervals()); + final Map locks = Tasks.tryAcquireExclusiveLocks(toolbox.getTaskActionClient(), intervals); + versions = locks.entrySet().stream() + .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); + + dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( + ingestionSchema.getDataSchema() + .getGranularitySpec() + .withIntervals( + JodaUtils.condenseIntervals( + shardSpecs.getIntervals() + ) + ) + ); + } else { + versions = getTaskLocks(toolbox.getTaskActionClient()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + dataSchema = ingestionSchema.getDataSchema(); + } + + final List pushedSegments = generateAndPushSegments( + toolbox, + dataSchema, + shardSpecs, + versions, + firehoseFactory, + firehoseTempDir + ); + + return TaskStatus.success(getId(), new TaskReport(getId(), pushedSegments)); + } + + private static boolean isGuaranteedRollup(IndexTask.IndexIOConfig ioConfig, IndexTask.IndexTuningConfig tuningConfig) + { + Preconditions.checkState( + !(tuningConfig.isForceGuaranteedRollup() && + (tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting())), + "Perfect rollup cannot be guaranteed with extendable shardSpecs" + ); + return tuningConfig.isForceGuaranteedRollup(); + } + + /** + * Determines intervals and shardSpecs for input data. This method first checks that it must determine intervals and + * shardSpecs by itself. Intervals must be determined if they are not specified in {@link GranularitySpec}. + * ShardSpecs must be determined if the perfect rollup must be guaranteed even though the number of shards is not + * specified in {@link IndexTask.IndexTuningConfig}. + *

+ * If both intervals and shardSpecs don't have to be determined, this method simply returns {@link IndexTask.ShardSpecs} for the + * given intervals. Here, if {@link IndexTask.IndexTuningConfig#numShards} is not specified, {@link NumberedShardSpec} is used. + *

+ * If one of intervals or shardSpecs need to be determined, this method reads the entire input for determining one of + * them. If the perfect rollup must be guaranteed, {@link HashBasedNumberedShardSpec} is used for hash partitioning + * of input data. In the future we may want to also support single-dimension partitioning. + * + * @return generated {@link IndexTask.ShardSpecs} representing a map of intervals and corresponding shard specs + */ + private IndexTask.ShardSpecs determineShardSpecs( + final FirehoseFactory firehoseFactory, + final File firehoseTempDir + ) throws IOException + { + final IndexTask.IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); + + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + + // Must determine intervals if unknown, since we acquire all locks before processing any data. + final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent(); + + // Must determine partitions if # of shards is not provided. + final boolean determineNumPartitions = tuningConfig.getNumShards() == null; + + // if we were given number of shards per interval and the intervals, we don't need to scan the data + if (!determineNumPartitions && !determineIntervals) { + log.info("Skipping determine partition scan"); + return createShardSpecWithoutInputScan(granularitySpec); + } else { + // determine intervals containing data + return createShardSpecsFromInput( + ingestionSchema, + firehoseFactory, + firehoseTempDir, + granularitySpec, + determineIntervals + ); + } + } + + private static IndexTask.ShardSpecs createShardSpecWithoutInputScan(GranularitySpec granularitySpec) + { + final Map> shardSpecs = new HashMap<>(); + final SortedSet intervals = granularitySpec.bucketIntervals().get(); + + for (Interval interval : intervals) { + shardSpecs.put(interval, ImmutableList.of()); + } + + return new IndexTask.ShardSpecs(shardSpecs); + } + + private static IndexTask.ShardSpecs createShardSpecsFromInput( + IndexTask.IndexIngestionSpec ingestionSchema, + FirehoseFactory firehoseFactory, + File firehoseTempDir, + GranularitySpec granularitySpec, + boolean determineIntervals + ) throws IOException + { + log.info("Determining intervals"); + long determineShardSpecsStartMillis = System.currentTimeMillis(); + + final List intervals = collectIntervalsAndShardSpecs( + ingestionSchema, + firehoseFactory, + firehoseTempDir, + granularitySpec, + determineIntervals + ); + + final Map> intervalToShardSpecs = intervals + .stream() + .collect(Collectors.toMap(Function.identity(), i -> ImmutableList.of())); + log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); + + return new IndexTask.ShardSpecs(intervalToShardSpecs); + } + + private static List collectIntervalsAndShardSpecs( + IndexTask.IndexIngestionSpec ingestionSchema, + FirehoseFactory firehoseFactory, + File firehoseTempDir, + GranularitySpec granularitySpec, + boolean determineIntervals + ) throws IOException + { + final List intervals = new ArrayList<>(); + int thrownAway = 0; + int unparseable = 0; + + try ( + final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser(), firehoseTempDir) + ) { + while (firehose.hasMore()) { + try { + final InputRow inputRow = firehose.nextRow(); + + // The null inputRow means the caller must skip this row. + if (inputRow == null) { + continue; + } + + final Interval interval; + if (determineIntervals) { + interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); + } else { + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + if (!optInterval.isPresent()) { + thrownAway++; + continue; + } + interval = optInterval.get(); + } + + if (!intervals.contains(interval)) { + intervals.add(interval); + } + } + catch (ParseException e) { + if (ingestionSchema.getTuningConfig().isReportParseExceptions()) { + throw e; + } else { + unparseable++; + } + } + } + } + + // These metrics are reported in generateAndPushSegments() + if (thrownAway > 0) { + log.warn("Unable to find a matching interval for [%,d] events", thrownAway); + } + if (unparseable > 0) { + log.warn("Unable to parse [%,d] events", unparseable); + } + + intervals.sort(Comparators.intervalsByStartThenEnd()); + return intervals; + } + + /** + * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}. + * If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs + * if one of below conditions are satisfied. + * + *

    + *
  • + * If the number of rows in a segment exceeds {@link IndexTask.IndexTuningConfig#targetPartitionSize} + *
  • + *
  • + * If the number of rows added to {@link BaseAppenderatorDriver} so far exceeds {@link IndexTask.IndexTuningConfig#maxTotalRows} + *
  • + *
+ * + * At the end of this method, all the remaining segments are published. + * + * @return true if generated segments are successfully published, otherwise false + */ + private List generateAndPushSegments( + final TaskToolbox toolbox, + final DataSchema dataSchema, + final ShardSpecs shardSpecs, + final Map versions, + final FirehoseFactory firehoseFactory, + final File firehoseTempDir + ) throws IOException, InterruptedException + { + final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + dataSchema, new RealtimeIOConfig(null, null, null), null + ); + final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + + if (toolbox.getMonitorScheduler() != null) { + toolbox.getMonitorScheduler().addMonitor( + new RealtimeMetricsMonitor( + ImmutableList.of(fireDepartmentForMetrics), + ImmutableMap.of(DruidMetrics.TASK_ID, new String[]{getId()}) + ) + ); + } + + final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); + final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); + final long pushTimeout = tuningConfig.getPushTimeout(); + + final SegmentAllocator segmentAllocator; + if (ioConfig.isAppendToExisting()) { + segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); + } else { + segmentAllocator = new CountingActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + getDataSource(), + granularitySpec, + shardSpecs.getMap(), + versions + ); + } + + try ( + final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); + final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); + final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) + ) { + driver.startJob(); + + final List pushedSegments = new ArrayList<>(); + + while (firehose.hasMore()) { + try { + final InputRow inputRow = firehose.nextRow(); + + if (inputRow == null) { + fireDepartmentMetrics.incrementThrownAway(); + continue; + } + + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + if (!optInterval.isPresent()) { + fireDepartmentMetrics.incrementThrownAway(); + continue; + } + + // Segments are created as needed, using a single sequence name. They may be allocated from the overlord + // (in append mode) or may be created on our own authority (in overwrite mode). + final String sequenceName = getId(); + final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); + + if (addResult.isOk()) { + if (exceedMaxRowsInSegment(addResult.getNumRowsInSegment(), tuningConfig) || + exceedMaxRowsInAppenderator(addResult.getTotalNumRowsInAppenderator(), tuningConfig)) { + // There can be some segments waiting for being published even though any rows won't be added to them. + // If those segments are not published here, the available space in appenderator will be kept to be small + // which makes the size of segments smaller. + final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); + pushedSegments.addAll(pushed.getSegments()); + log.info("Pushed segments[%s]", pushed.getSegments()); + } + } else { + throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); + } + + fireDepartmentMetrics.incrementProcessed(); + } + catch (ParseException e) { + if (tuningConfig.isReportParseExceptions()) { + throw e; + } else { + fireDepartmentMetrics.incrementUnparseable(); + } + } + } + + final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); + pushedSegments.addAll(pushed.getSegments()); + log.info("Pushed segments[%s]", pushed.getSegments()); + + return pushedSegments; + } + catch (TimeoutException | ExecutionException e) { + throw Throwables.propagate(e); + } + } + + private static boolean exceedMaxRowsInSegment(int numRowsInSegment, IndexTuningConfig indexTuningConfig) + { + // maxRowsInSegment should be null if numShards is set in indexTuningConfig + final Integer maxRowsInSegment = indexTuningConfig.getTargetPartitionSize(); + return maxRowsInSegment != null && maxRowsInSegment <= numRowsInSegment; + } + + private static boolean exceedMaxRowsInAppenderator(long numRowsInAppenderator, IndexTuningConfig indexTuningConfig) + { + // maxRowsInAppenderator should be null if numShards is set in indexTuningConfig + final Long maxRowsInAppenderator = indexTuningConfig.getMaxTotalRows(); + return maxRowsInAppenderator != null && maxRowsInAppenderator <= numRowsInAppenderator; + } + + private static Appenderator newAppenderator( + FireDepartmentMetrics metrics, + TaskToolbox toolbox, + DataSchema dataSchema, + IndexTuningConfig tuningConfig + ) + { + return Appenderators.createOffline( + dataSchema, + tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + toolbox.getSegmentPusher(), + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9() + ); + } + + private static BatchAppenderatorDriver newDriver( + final Appenderator appenderator, + final TaskToolbox toolbox, + final SegmentAllocator segmentAllocator + ) + { + return new BatchAppenderatorDriver( + appenderator, + segmentAllocator, + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()) + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java new file mode 100644 index 000000000000..265ab22eb730 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java @@ -0,0 +1,362 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterators; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.TaskStatus; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.indexer.TaskState; +import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.IndexTask.IndexIOConfig; +import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; +import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import io.druid.segment.realtime.appenderator.UsedSegmentChecker; +import io.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * ParallelIndexSinglePhaseSupervisorTask is capable of running multiple subTasks for parallel indexing. This is + * applicable if the input {@link FiniteFirehoseFactory} is splittable. While this task is running, it can submit + * multiple child tasks to overlords. This task succeeds only when all its child tasks succeed; otherwise it fails. + * + * As its name indicates, distributed indexing is done in a single phase, i.e., without shuffling intermediate data. As + * a result, this task can't be used for perfect rollup. + */ +public class ParallelIndexSinglePhaseSupervisorTask extends AbstractTask +{ + private static final Logger log = new Logger(ParallelIndexSinglePhaseSupervisorTask.class); + private static final String TYPE = "parallelIndexSinglePhase"; + + private final ParallelIndexSinglePhaseIngestionSpec ingestionSchema; + private final FiniteFirehoseFactory baseFirehoseFactory; + private final int maxNumTasks; + private final TaskMonitor taskMonitor; + + private final BlockingQueue taskCompleteEvents = new LinkedBlockingDeque<>(); + private final List segments = new ArrayList<>(); + + private volatile boolean stopped; + + @JsonCreator + public ParallelIndexSinglePhaseSupervisorTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("spec") ParallelIndexSinglePhaseIngestionSpec ingestionSchema, + @JsonProperty("context") Map context, + @JacksonInject IndexingServiceClient indexingServiceClient + ) + { + super( + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + null, + taskResource, + ingestionSchema.getDataSchema().getDataSource(), + context + ); + + this.ingestionSchema = ingestionSchema; + + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + if (!(firehoseFactory instanceof FiniteFirehoseFactory)) { + throw new IAE("[%s] should implement FiniteFirehoseFactory", firehoseFactory.getClass().getSimpleName()); + } + + this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; + this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumBatchTasks(); + this.taskMonitor = new TaskMonitor(indexingServiceClient, ingestionSchema.getTuningConfig().getMaxRetry()); + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); + } + + @Override + public String getType() + { + return TYPE; + } + + @JsonProperty("spec") + ParallelIndexSinglePhaseIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + final Optional> intervals = ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals(); + + return !intervals.isPresent() || isReady(taskActionClient, intervals.get()); + } + + static boolean isReady(TaskActionClient actionClient, SortedSet intervals) throws IOException + { + final List locks = getTaskLocks(actionClient); + if (locks.size() == 0) { + try { + Tasks.tryAcquireExclusiveLocks(actionClient, intervals); + } + catch (Exception e) { + log.error(e, "Failed to acquire locks for intervals[%s]", intervals); + return false; + } + } + return true; + } + + @Override + public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Exception + { + if (baseFirehoseFactory.isSplittable()) { + return runParallel(toolbox); + } else { + log.warn( + "firehoseFactory[%s] is not splittable. Running sequentially", + baseFirehoseFactory.getClass().getSimpleName() + ); + return runSequential(toolbox); + } + } + + private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) throws Exception + { + final Iterator subTaskIterator = subTaskIterator(); + final int numTotalTasks = baseFirehoseFactory.getNumSplits(); + final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckingPeriodMs(); + TaskState state = TaskState.FAILED; + + log.info("Total number of tasks is [%d]", numTotalTasks); + + int numCompleteTasks = 0; + taskMonitor.start(taskStatusCheckingPeriod); + + try { + log.info("Submitting initial tasks"); + // Submit initial tasks + while (subTaskIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) { + submitNewTask(subTaskIterator.next()); + } + + while (!stopped && !Thread.currentThread().isInterrupted()) { + final TaskStatus taskCompleteEvent = taskCompleteEvents.poll(taskStatusCheckingPeriod, TimeUnit.MILLISECONDS); + + if (taskCompleteEvent != null) { + final TaskState completeState = taskCompleteEvent.getStatusCode(); + if (completeState == null) { + throw new ISE("Complete state of task[%s] is null", taskCompleteEvent.getId()); + } + switch (completeState) { + case SUCCESS: + numCompleteTasks++; + segments.addAll((Collection) taskCompleteEvent.getReport().getPayload()); + log.info("[%d/%d] tasks succeeded", numCompleteTasks, numTotalTasks); + if (!subTaskIterator.hasNext()) { + if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) { + stopped = true; + if (numCompleteTasks == numTotalTasks) { + // Publishing all segments reported so far + publish(toolbox); + + // Succeeded + state = TaskState.SUCCESS; + } else { + // Failed + throw new ISE("Expected to complete [%d] tasks, but we got [%d]", numTotalTasks, numCompleteTasks); + } + } + } else if (taskMonitor.getNumRunningTasks() < maxNumTasks) { + submitNewTask(subTaskIterator.next()); + } + break; + case FAILED: + // TaskMonitor already tried everything it can do for failed tasks. We failed. + state = TaskState.FAILED; + stopped = true; + log.error("Failed because of the failed sub task[%s]", taskCompleteEvent.getId()); + break; + default: + throw new ISE("Complete task[%s] is in an invalid state[%s]", taskCompleteEvent.getId(), completeState); + } + } + } + } + finally { + // Cleanup resources + taskCompleteEvents.clear(); + taskMonitor.stop(); + + if (state != TaskState.SUCCESS) { + // if this fails, kill all sub tasks + taskMonitor.killAll(); + } + } + + return io.druid.indexing.common.TaskStatus.fromCode(getId(), state); + } + + private io.druid.indexing.common.TaskStatus runSequential(TaskToolbox toolbox) throws Exception + { + return new IndexTask( + getId(), + getGroupId(), + getTaskResource(), + getDataSource(), + new IndexIngestionSpec( + getIngestionSchema().getDataSchema(), + getIngestionSchema().getIOConfig(), + convertToIndexTuningConfig(getIngestionSchema().getTuningConfig()) + ), + getContext() + ).run(toolbox); + } + + private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexSinglePhaseTuningConfig tuningConfig) + { + return new IndexTuningConfig( + tuningConfig.getTargetPartitionSize(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxTotalRows(), + null, + tuningConfig.getNumShards(), + tuningConfig.getIndexSpec(), + tuningConfig.getMaxPendingPersists(), + true, + tuningConfig.isForceExtendableShardSpecs(), + tuningConfig.isForceGuaranteedRollup(), + tuningConfig.isReportParseExceptions(), + null, + tuningConfig.getPushTimeout(), + tuningConfig.getSegmentWriteOutMediumFactory() + ); + } + + private void publish(TaskToolbox toolbox) throws IOException + { + final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); + return toolbox.getTaskActionClient().submit(action).isSuccess(); + }; + final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()); + final Set segmentsToPublish = ImmutableSet.copyOf(segments); + final boolean published = publisher.publishSegments(segmentsToPublish, null); + + if (published) { + log.info("Published segments"); + } else { + log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); + final Set segmentsIdentifiers = segments + .stream() + .map(SegmentIdentifier::fromDataSegment) + .collect(Collectors.toSet()); + if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers) + .equals(segmentsToPublish)) { + log.info("Our segments really do exist, awaiting handoff."); + } else { + throw new ISE("Failed to publish segments[%s]", segmentsToPublish); + } + } + } + + private void submitNewTask(ParallelIndexSinglePhaseSubTask task) + { + final ListenableFuture future = taskMonitor.submit(task); + Futures.addCallback( + future, + new FutureCallback() + { + @Override + public void onSuccess(TaskStatus taskStatus) + { + taskCompleteEvents.offer(taskStatus); + } + + @Override + public void onFailure(Throwable t) + { + taskCompleteEvents.offer(new TaskStatus(task.getId(), TaskState.FAILED, null, -1)); + } + } + ); + } + + @VisibleForTesting + Iterator subTaskIterator() throws IOException + { + return Iterators.transform(baseFirehoseFactory.getSplits(), split -> newTask((InputSplit) split)); + } + + @VisibleForTesting + ParallelIndexSinglePhaseSubTask newTask(InputSplit split) + { + return new ParallelIndexSinglePhaseSubTask( + null, + getGroupId(), + null, + new IndexIngestionSpec( + ingestionSchema.getDataSchema(), + new IndexIOConfig( + baseFirehoseFactory.withSplit(split), + ingestionSchema.getIOConfig().isAppendToExisting() + ), + convertToIndexTuningConfig(ingestionSchema.getTuningConfig()) + ), + getContext() + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java new file mode 100644 index 000000000000..b60c2759de87 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java @@ -0,0 +1,176 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.segment.IndexSpec; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; + +import javax.annotation.Nullable; +import java.util.Objects; + +@JsonTypeName("parallelIndexSinglePhase") +public class ParallelIndexSinglePhaseTuningConfig extends IndexTuningConfig +{ + private static final int DEFAULT_MAX_NUM_BATCH_TASKS = Integer.MAX_VALUE; // unlimited + private static final int DEFAULT_MAX_RETRY = 3; + private static final long DEFAULT_TASK_STATUS_CHECKING_PERIOD_MS = 1000; + + private final int maxNumBatchTasks; + private final int maxRetry; + private final long taskStatusCheckingPeriodMs; + + public static ParallelIndexSinglePhaseTuningConfig defaultConfig() + { + return new ParallelIndexSinglePhaseTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } + + @JsonCreator + public ParallelIndexSinglePhaseTuningConfig( + @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize, + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, + @JsonProperty("rowFlushBoundary") @Nullable Integer rowFlushBoundary_forBackCompatibility, // DEPRECATED + @JsonProperty("numShards") @Nullable Integer numShards, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. + @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, + @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs, + @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("publishTimeout") @Nullable Long publishTimeout, // deprecated + @JsonProperty("pushTimeout") @Nullable Long pushTimeout, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("maxNumBatchTasks") @Nullable Integer maxNumBatchTasks, + @JsonProperty("maxRetry") @Nullable Integer maxRetry, + @JsonProperty("taskStatusCheckingPeriodMs") @Nullable Integer taskStatusCheckingPeriodMs + ) + { + super( + targetPartitionSize, + maxRowsInMemory, + maxTotalRows, + rowFlushBoundary_forBackCompatibility, + numShards, + indexSpec, + maxPendingPersists, + buildV9Directly, + forceExtendableShardSpecs, + false, // ParallelIndexSinglePhaseSupervisorTask can't be used for guaranteed rollup + reportParseExceptions, + publishTimeout, + pushTimeout, + segmentWriteOutMediumFactory + ); + + this.maxNumBatchTasks = maxNumBatchTasks == null ? DEFAULT_MAX_NUM_BATCH_TASKS : maxNumBatchTasks; + this.maxRetry = maxRetry == null ? DEFAULT_MAX_RETRY : maxRetry; + this.taskStatusCheckingPeriodMs = taskStatusCheckingPeriodMs == null ? + DEFAULT_TASK_STATUS_CHECKING_PERIOD_MS : + taskStatusCheckingPeriodMs; + } + + @JsonProperty + public int getMaxNumBatchTasks() + { + return maxNumBatchTasks; + } + + @JsonProperty + public int getMaxRetry() + { + return maxRetry; + } + + @JsonProperty + public long getTaskStatusCheckingPeriodMs() + { + return taskStatusCheckingPeriodMs; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ParallelIndexSinglePhaseTuningConfig that = (ParallelIndexSinglePhaseTuningConfig) o; + return getMaxRowsInMemory() == that.getMaxRowsInMemory() && + Objects.equals(getMaxTotalRows(), that.getMaxTotalRows()) && + getMaxPendingPersists() == that.getMaxPendingPersists() && + isForceExtendableShardSpecs() == that.isForceExtendableShardSpecs() && + isForceGuaranteedRollup() == that.isForceGuaranteedRollup() && + isReportParseExceptions() == that.isReportParseExceptions() && + getPushTimeout() == that.getPushTimeout() && + Objects.equals(getTargetPartitionSize(), that.getTargetPartitionSize()) && + Objects.equals(getNumShards(), that.getNumShards()) && + Objects.equals(getIndexSpec(), that.getIndexSpec()) && + Objects.equals(getBasePersistDirectory(), that.getBasePersistDirectory()) && + Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()) && + maxNumBatchTasks == that.maxNumBatchTasks && + maxRetry == that.maxRetry && + taskStatusCheckingPeriodMs == that.taskStatusCheckingPeriodMs; + } + + @Override + public int hashCode() + { + return Objects.hash( + getTargetPartitionSize(), + getMaxRowsInMemory(), + getMaxTotalRows(), + getNumShards(), + getIndexSpec(), + getBasePersistDirectory(), + getMaxPendingPersists(), + isForceExtendableShardSpecs(), + isForceGuaranteedRollup(), + isReportParseExceptions(), + getPushTimeout(), + getSegmentWriteOutMediumFactory(), + maxNumBatchTasks, + maxRetry, + taskStatusCheckingPeriodMs + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index e3eb3e77c5e5..d8e3d2e61fdb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -51,6 +51,8 @@ @JsonSubTypes.Type(name = "archive", value = ArchiveTask.class), @JsonSubTypes.Type(name = "restore", value = RestoreTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), + @JsonSubTypes.Type(name = "parallelIndexSinglePhase", value = ParallelIndexSinglePhaseSupervisorTask.class), + @JsonSubTypes.Type(name = "subIndexSinglePhase", value = ParallelIndexSinglePhaseSubTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment", value = HadoopConverterTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment_sub", value = HadoopConverterTask.ConverterSubTask.class), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java new file mode 100644 index 000000000000..cae69a2841ed --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -0,0 +1,192 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.SettableFuture; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.TaskStatus; +import io.druid.client.indexing.TaskStatusResponse; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.logger.Logger; + +import java.util.Iterator; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Responsible for submitting tasks, monitoring task statuses, resubmitting failed tasks, and returning the final task + * status. + */ +public class TaskMonitor +{ + private static final Logger log = new Logger(TaskMonitor.class); + + private final ScheduledExecutorService taskStatusChecker = Execs.scheduledSingleThreaded(("task-monitor-%d")); + + private final ConcurrentMap taskFutureMap = new ConcurrentHashMap<>(); + + // overlord client + private final AtomicInteger numRunningTasks = new AtomicInteger(); + + private final IndexingServiceClient indexingServiceClient; + private final int maxRetry; + + private volatile boolean running = false; + + TaskMonitor(IndexingServiceClient indexingServiceClient, int maxRetry) + { + this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"); + this.maxRetry = maxRetry; + } + + public void start(long taskStatusCheckingPeriod) + { + running = true; + log.info("Starting taskMonitor"); + // NOTE: This polling can be improved to event-driven processing by registering TaskRunnerListener to TaskRunner. + // That listener should be able to send the events reported to TaskRunner to this TaskMonitor. + taskStatusChecker.scheduleAtFixedRate( + () -> { + try { + final Iterator> iterator = taskFutureMap.entrySet().iterator(); + while (iterator.hasNext()) { + final Entry entry = iterator.next(); + final String taskId = entry.getKey(); + final MonitorEntry monitorEntry = entry.getValue(); + final TaskStatusResponse taskStatusResponse = indexingServiceClient.getTaskStatus(taskId); + if (taskStatusResponse != null) { + final TaskStatus taskStatus = taskStatusResponse.getStatus(); + switch (taskStatus.getStatusCode()) { + case SUCCESS: + numRunningTasks.decrementAndGet(); + iterator.remove(); + monitorEntry.setLastStatus(taskStatus); + break; + case FAILED: + numRunningTasks.decrementAndGet(); + log.warn("task[%s] failed!", taskId); + if (monitorEntry.numRetry < maxRetry) { + log.info( + "We still have chnaces[%d/%d]. Retrying task[%s]", + monitorEntry.numRetry, + maxRetry, + taskId + ); + retry(monitorEntry.task); + monitorEntry.incrementNumRetry(); + } else { + log.error( + "task[%s] failed after [%d] retries", + taskId, + monitorEntry.numRetry + ); + iterator.remove(); + monitorEntry.setLastStatus(taskStatus); + } + break; + default: + // do nothing + } + } + } + } + catch (Throwable t) { + log.error(t, "Error while monitoring"); + } + }, + taskStatusCheckingPeriod, + taskStatusCheckingPeriod, + TimeUnit.MILLISECONDS + ); + } + + public void stop() + { + running = false; + taskStatusChecker.shutdownNow(); + log.info("Stopped taskMonitor"); + } + + public ListenableFuture submit(Task task) + { + if (!running) { + return Futures.immediateFailedFuture(new ISE("TaskMonitore is not running")); + } + log.info("Submitting a new task[%s]", task.getId()); + final String taskId = indexingServiceClient.runTask(task); + + numRunningTasks.incrementAndGet(); + + final SettableFuture taskFuture = SettableFuture.create(); + taskFutureMap.put(taskId, new MonitorEntry(task, taskFuture)); + return taskFuture; + } + + private void retry(Task task) + { + if (running) { + indexingServiceClient.runTask(task); + numRunningTasks.incrementAndGet(); + } + } + + public void killAll() + { + taskFutureMap.keySet().forEach(indexingServiceClient::killTask); + taskFutureMap.clear(); + } + + public int getNumRunningTasks() + { + return numRunningTasks.intValue(); + } + + private static class MonitorEntry + { + private final Task task; + private final SettableFuture future; + + private int numRetry; + + MonitorEntry(Task task, SettableFuture future) + { + this.task = task; + this.future = future; + } + + void setLastStatus(TaskStatus taskStatus) + { + future.set(taskStatus); + } + + void incrementNumRetry() + { + numRetry++; + } + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index a5cde1656476..708c12c7ed26 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -31,7 +31,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; -import io.druid.java.util.emitter.EmittingLogger; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.task.Task; @@ -39,6 +38,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.emitter.EmittingLogger; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -890,8 +890,20 @@ TaskLock getTaskLock() boolean addTask(Task task) { - Preconditions.checkArgument(taskLock.getGroupId().equals(task.getGroupId())); - Preconditions.checkArgument(taskLock.getPriority() == task.getPriority()); + Preconditions.checkArgument( + taskLock.getGroupId().equals(task.getGroupId()), + "groupId[%s] of task[%s] is different from the existing lockPosse's groupId[%s]", + task.getGroupId(), + task.getId(), + taskLock.getGroupId() + ); + Preconditions.checkArgument( + taskLock.getPriority() == task.getPriority(), + "priority[%s] of task[%s] is different from the existing lockPosse's priority[%s]", + task.getPriority(), + task.getId(), + taskLock.getPriority() + ); return taskIds.add(task.getId()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java index f1d14e10f2b0..4036249b4c83 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java @@ -22,8 +22,6 @@ import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.inject.Inject; -import io.druid.java.util.emitter.EmittingLogger; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.client.indexing.IndexingService; import io.druid.curator.discovery.ServiceAnnouncer; import io.druid.discovery.DruidLeaderSelector; @@ -38,6 +36,8 @@ import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.server.DruidNode; import io.druid.server.coordinator.CoordinatorOverlordServiceConfig; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java new file mode 100644 index 000000000000..9f83e7ba3893 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java @@ -0,0 +1,149 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.druid.indexing.common.task.IngestionTestBase; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.Task; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class CountingSegmentAllocateActionTest extends IngestionTestBase +{ + @Test + public void testJsonSerde() throws IOException + { + final SimpleModule module = new SimpleModule(); + module.addSerializer(Interval.class, ToStringSerializer.instance); + + final ObjectMapper mapper = new DefaultObjectMapper(); + mapper.registerModule(module); + mapper.registerSubtypes(new NamedType(NumberedShardSpec.class, "numbered")); + + final CountingSegmentAllocateAction action = createAction(DateTimes.nowUtc()); + + final String json = mapper.writeValueAsString(action); + Assert.assertEquals(action, mapper.readValue(json, TaskAction.class)); + } + + @Test + public void testPerform() + { + final Task task = NoopTask.create(); + final TaskActionToolbox toolbox = createTaskActionToolbox(); + + CountingSegmentAllocateAction action; + SegmentIdentifier segmentIdentifier; + + for (int i = 0; i < 3; i++) { + action = createAction(DateTimes.of("2017-01-01")); + segmentIdentifier = action.perform(task, toolbox); + Assert.assertEquals( + StringUtils.format("dataSource_2017-01-01T00:00:00.000Z_2017-01-02T00:00:00.000Z_version1_%d", i + 1), + segmentIdentifier.toString() + ); + } + + for (int i = 0; i < 3; i++) { + action = createAction(DateTimes.of("2017-01-02")); + segmentIdentifier = action.perform(task, toolbox); + Assert.assertEquals( + StringUtils.format("dataSource_2017-01-02T00:00:00.000Z_2017-01-03T00:00:00.000Z_version1_%d", i + 1), + segmentIdentifier.toString() + ); + } + } + + @Test + public void testPerformParallel() + { + final List tasks = IntStream.range(0, 5).mapToObj(i -> NoopTask.create()).collect(Collectors.toList()); + final ExecutorService service = Execs.multiThreaded(5, "counting-segment-allocate-action-test-%d"); + final TaskActionToolbox toolbox = createTaskActionToolbox(); + + try { + final List segmentIdentifiers = tasks + .stream() + .map(task -> { + final CountingSegmentAllocateAction action = createAction(DateTimes.of("2017-01-01")); + return action.perform(task, toolbox); + }) + .sorted(Comparator.comparing(SegmentIdentifier::toString)) + .collect(Collectors.toList()); + for (int i = 0; i < 5; i++) { + Assert.assertEquals( + StringUtils.format("dataSource_2017-01-01T00:00:00.000Z_2017-01-02T00:00:00.000Z_version1_%d", i + 1), + segmentIdentifiers.get(i).toString() + ); + } + } + finally { + service.shutdownNow(); + } + } + + private CountingSegmentAllocateAction createAction(DateTime timestamp) + { + return new CountingSegmentAllocateAction( + "dataSource", + timestamp, + new ArbitraryGranularitySpec( + Granularities.DAY, + ImmutableList.of(Intervals.of("2017-01-01/2017-01-02"), Intervals.of("2017-01-02/2017-01-03")) + ), + ImmutableMap.of( + Intervals.of("2017-01-01/2017-01-02"), + ImmutableList.of(new NumberedShardSpec(0, 0)), + Intervals.of("2017-01-02/2017-01-03"), + ImmutableList.of(new NumberedShardSpec(1, 0)) + ), + ImmutableMap.of( + Intervals.of("2017-01-01/2017-01-02"), + "version1", + Intervals.of("2017-01-02/2017-01-03"), + "version1" + ) + ); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java index 2cabd2c2c1c0..72780efdc73a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; +import io.druid.indexing.common.Counters; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.overlord.HeapMemoryTaskStorage; @@ -111,7 +112,8 @@ public void before() taskLockbox, metadataStorageCoordinator, new NoopServiceEmitter(), - EasyMock.createMock(SupervisorManager.class) + EasyMock.createMock(SupervisorManager.class), + new Counters() ); testDerbyConnector.createDataSourceTable(); testDerbyConnector.createPendingSegmentsTable(); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index cfa91b1a2205..a607ebfe74bb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -42,6 +42,7 @@ import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; import io.druid.indexer.TaskState; +import io.druid.indexing.common.Counters; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -1138,7 +1139,8 @@ public SegmentPublishResult announceHistoricalSegments( taskLockbox, mdc, emitter, - EasyMock.createMock(SupervisorManager.class) + EasyMock.createMock(SupervisorManager.class), + new Counters() ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java new file mode 100644 index 000000000000..c3da4ae07630 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java @@ -0,0 +1,107 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.indexing.common.Counters; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TestUtils; +import io.druid.indexing.common.actions.LocalTaskActionClient; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.actions.TaskActionToolbox; +import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.metadata.EntryExistsException; +import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import io.druid.metadata.TestDerbyConnector; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMergerV9; +import io.druid.server.metrics.NoopServiceEmitter; +import org.junit.Rule; + +public abstract class IngestionTestBase +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private final TestUtils testUtils = new TestUtils(); + private final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + private final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); + private final TaskLockbox lockbox = new TaskLockbox(taskStorage); + + public IngestionTestBase() + { + } + + public TaskActionClient createActionClient(Task task) + { + return new LocalTaskActionClient(task, taskStorage, createTaskActionToolbox()); + } + + public void prepareTaskForLocking(Task task) throws EntryExistsException + { + lockbox.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public TaskStorage getTaskStorage() + { + return taskStorage; + } + + public TaskLockbox getLockbox() + { + return lockbox; + } + + public TaskActionToolbox createTaskActionToolbox() + { + final IndexerSQLMetadataStorageCoordinator storageCoordinator = new IndexerSQLMetadataStorageCoordinator( + objectMapper, + derbyConnectorRule.metadataTablesConfigSupplier().get(), + derbyConnectorRule.getConnector() + ); + storageCoordinator.start(); + return new TaskActionToolbox( + lockbox, + storageCoordinator, + new NoopServiceEmitter(), + null, + new Counters() + ); + } + + public IndexIO getIndexIO() + { + return testUtils.getTestIndexIO(); + } + + public IndexMergerV9 getIndexMerger() + { + return testUtils.getTestIndexMergerV9(); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java new file mode 100644 index 000000000000..72d7ce7bc1e6 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java @@ -0,0 +1,392 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.google.common.base.Throwables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.TaskStatusResponse; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.CSVParseSpec; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.ParseSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.indexer.TaskState; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.LocalDataSegmentPusher; +import io.druid.segment.loading.LocalDataSegmentPusherConfig; +import io.druid.segment.realtime.firehose.LocalFirehoseFactory; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +public class ParallelIndexSinglePhaseSupervisorTaskTest extends IngestionTestBase +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( + new TimestampSpec( + "ts", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("ts", "dim", "val"), + false, + 0 + ); + + private TaskActionClient actionClient; + private LocalIndexingServiceClient indexingServiceClient; + private TaskToolbox toolbox; + private File localDeepStorage; + private File inputDir; + + @Before + public void setup() throws IOException + { + inputDir = temporaryFolder.newFolder("data"); + // set up data + for (int i = 0; i < 5; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 25 + i, i)); + } + } + + for (int i = 0; i < 5; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "filtered_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 25 + i, i)); + } + } + + indexingServiceClient = new LocalIndexingServiceClient(); + localDeepStorage = temporaryFolder.newFolder("localStorage"); + } + + @After + public void teardown() + { + indexingServiceClient.shutdown(); + temporaryFolder.delete(); + } + + @Test + public void testIsReady() throws Exception + { + final ParallelIndexSinglePhaseSupervisorTask task = newTask( + new ParallelIndexSinglePhaseIOConfig( + new LocalFirehoseFactory(inputDir, "test_*", null), + false + ) + ); + actionClient = createActionClient(task); + toolbox = createTaskToolbox(task); + + prepareTaskForLocking(task); + Assert.assertTrue(task.isReady(actionClient)); + + final Iterator subTaskIterator = task.subTaskIterator(); + + while (subTaskIterator.hasNext()) { + final ParallelIndexSinglePhaseSubTask subTask = subTaskIterator.next(); + final TaskActionClient subTaskActionClient = createActionClient(subTask); + prepareTaskForLocking(subTask); + Assert.assertTrue(subTask.isReady(subTaskActionClient)); + } + } + + @Test() + public void testRunInParallel() throws Exception + { + final ParallelIndexSinglePhaseSupervisorTask task = newTask( + new ParallelIndexSinglePhaseIOConfig( + new LocalFirehoseFactory(inputDir, "test_*", null), + false + ) + ); + actionClient = createActionClient(task); + toolbox = createTaskToolbox(task); + + prepareTaskForLocking(task); + Assert.assertTrue(task.isReady(actionClient)); + Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); + } + + @Test + public void testRunInSequential() throws Exception + { + final ParallelIndexSinglePhaseSupervisorTask task = newTask( + new ParallelIndexSinglePhaseIOConfig( + new LocalFirehoseFactory(inputDir, "test_*", null) + { + @Override + public boolean isSplittable() + { + return false; + } + }, + false + ) + ); + actionClient = createActionClient(task); + toolbox = createTaskToolbox(task); + + prepareTaskForLocking(task); + Assert.assertTrue(task.isReady(actionClient)); + Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); + } + + private ParallelIndexSinglePhaseSupervisorTask newTask(ParallelIndexSinglePhaseIOConfig ioConfig) + { + // set up ingestion spec + final ParallelIndexSinglePhaseIngestionSpec singlePhaseIngestionSpec = new ParallelIndexSinglePhaseIngestionSpec( + new DataSchema( + "dataSource", + getObjectMapper().convertValue( + new StringInputRowParser( + DEFAULT_PARSE_SPEC, + null + ), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Arrays.asList(Intervals.of("2017/2018")) + ), + null, + getObjectMapper() + ), + ioConfig, + null + ); + + // set up test tools + return new TestParallelIndexSinglePhaseSupervisorTask( + null, + null, + singlePhaseIngestionSpec, + new HashMap<>(), + indexingServiceClient + ); + } + + private class TestParallelIndexSinglePhaseSupervisorTask extends ParallelIndexSinglePhaseSupervisorTask + { + + TestParallelIndexSinglePhaseSupervisorTask( + String id, + TaskResource taskResource, + ParallelIndexSinglePhaseIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + super(id, taskResource, ingestionSchema, context, indexingServiceClient); + } + + @Override + Iterator subTaskIterator() throws IOException + { + final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() + .getIOConfig() + .getFirehoseFactory(); + return Iterators.transform(baseFirehoseFactory.getSplits(), split -> { + try { + // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id + Thread.sleep(10); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + return newTask((InputSplit) split); + }); + } + } + + private class LocalIndexingServiceClient extends IndexingServiceClient + { + private final ConcurrentMap> tasks = new ConcurrentHashMap<>(); + private final ListeningExecutorService service = MoreExecutors.listeningDecorator( + Execs.multiThreaded(5, "parallel-index-single-phase-supervisor-task-test-%d") + ); + + LocalIndexingServiceClient() + { + super(null, null); + } + + @Override + public String runTask(Object taskObject) + { + final ParallelIndexSinglePhaseSubTask subTask = (ParallelIndexSinglePhaseSubTask) taskObject; + tasks.put(subTask.getId(), service.submit(() -> { + try { + final TaskToolbox toolbox = createTaskToolbox(subTask); + if (subTask.isReady(toolbox.getTaskActionClient())) { + return subTask.run(toolbox); + } else { + throw new ISE("task[%s] is not ready", subTask.getId()); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + })); + return subTask.getId(); + } + + @Override + @Nullable + public TaskStatusResponse getTaskStatus(String taskId) + { + final Future taskStatusFuture = tasks.get(taskId); + if (taskStatusFuture != null) { + try { + if (taskStatusFuture.isDone()) { + final TaskStatus taskStatus = taskStatusFuture.get(); + return new TaskStatusResponse( + taskId, + new io.druid.client.indexing.TaskStatus(taskId, taskStatus.getStatusCode(), taskStatus.getReport(), -1) + ); + } else { + return new TaskStatusResponse( + taskId, + new io.druid.client.indexing.TaskStatus(taskId, TaskState.RUNNING, null, -1) + ); + } + } + catch (InterruptedException | ExecutionException e) { + // We don't have a way to pass this exception to the supervisorTask yet.. + // So, let's print it here. + System.err.println(Throwables.getStackTraceAsString(e)); + return new TaskStatusResponse( + taskId, + new io.druid.client.indexing.TaskStatus(taskId, TaskState.FAILED, null, -1) + ); + } + } else { + return null; + } + } + + @Override + public String killTask(String taskId) + { + final Future taskStatusFuture = tasks.remove(taskId); + if (taskStatusFuture != null) { + taskStatusFuture.cancel(true); + return taskId; + } else { + return null; + } + } + + void shutdown() + { + service.shutdownNow(); + } + } + + private TaskToolbox createTaskToolbox(Task task) throws IOException + { + return new TaskToolbox( + null, + actionClient, + null, + new LocalDataSegmentPusher( + new LocalDataSegmentPusherConfig() + { + @Override + public File getStorageDirectory() + { + return localDeepStorage; + } + }, + getObjectMapper() + ), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + getObjectMapper(), + temporaryFolder.newFolder(task.getId()), + getIndexIO(), + null, + null, + getIndexMerger(), + null, + null, + null, + null + ); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 9f050fdcab47..e491d15380a1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -44,6 +44,7 @@ import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; import io.druid.indexer.TaskState; +import io.druid.indexing.common.Counters; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -982,7 +983,8 @@ private TaskToolbox makeToolbox( taskLockbox, mdc, emitter, - EasyMock.createMock(SupervisorManager.class) + EasyMock.createMock(SupervisorManager.class), + new Counters() ); final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory( taskStorage, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java new file mode 100644 index 000000000000..986372fd7028 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java @@ -0,0 +1,146 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.google.common.util.concurrent.ListenableFuture; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.TaskStatus; +import io.druid.client.indexing.TaskStatusResponse; +import io.druid.indexer.TaskState; +import io.druid.indexing.common.TaskToolbox; +import io.druid.java.util.common.concurrent.Execs; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class TaskMonitorTest +{ + private final ExecutorService taskRunner = Execs.multiThreaded(5, "task-monitor-test-%d"); + private final ConcurrentMap tasks = new ConcurrentHashMap<>(); + private final TaskMonitor monitor = new TaskMonitor(new TestIndexingServiceClient(), 3); + + @Before + public void setup() + { + tasks.clear(); + monitor.start(100); + } + + @After + public void teardown() + { + monitor.stop(); + taskRunner.shutdownNow(); + } + + @Test + public void testBasic() throws InterruptedException, ExecutionException, TimeoutException + { + final List> futures = IntStream + .range(0, 10) + .mapToObj(i -> monitor.submit(new TestTask("id" + i, 100, 0))) + .collect(Collectors.toList()); + for (int i = 0; i < futures.size(); i++) { + // # of threads of taskRunner is 5, so the expected max timeout is 2 sec. We additionally wait three more seconds + // here to make sure the test passes. + final TaskStatus result = futures.get(i).get(1, TimeUnit.SECONDS); + Assert.assertEquals("id" + i, result.getId()); + Assert.assertEquals(TaskState.SUCCESS, result.getStatusCode()); + } + } + + @Test + public void testRetry() throws InterruptedException, ExecutionException, TimeoutException + { + final List> futures = IntStream + .range(0, 10) + .mapToObj(i -> monitor.submit(new TestTask("id" + i, 100, 2))) + .collect(Collectors.toList()); + for (int i = 0; i < futures.size(); i++) { + // # of threads of taskRunner is 5, and each task is expected to be run 3 times (with 2 retries), so the expected + // max timeout is 6 sec. We additionally wait 4 more seconds here to make sure the test passes. + final TaskStatus result = futures.get(i).get(2, TimeUnit.SECONDS); + Assert.assertEquals("id" + i, result.getId()); + Assert.assertEquals(TaskState.SUCCESS, result.getStatusCode()); + } + } + + private class TestTask extends NoopTask + { + private final int numMaxFails; + + private int numFails; + + public TestTask(String id, long runTime, int numMaxFails) + { + super(id, "testDataSource", runTime, 0, null, null, null); + this.numMaxFails = numMaxFails; + } + + @Override + public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Exception + { + if (numFails < numMaxFails) { + numFails++; + Thread.sleep(getRunTime()); + return io.druid.indexing.common.TaskStatus.failure(getId()); + } else { + return super.run(toolbox); + } + } + } + + private class TestIndexingServiceClient extends IndexingServiceClient + { + TestIndexingServiceClient() + { + super(null, null); + } + + @Override + public String runTask(Object taskObject) + { + final TestTask task = (TestTask) taskObject; + tasks.put(task.getId(), TaskState.RUNNING); + taskRunner.submit(() -> tasks.put(task.getId(), task.run(null).getStatusCode())); + return task.getId(); + } + + @Override + public TaskStatusResponse getTaskStatus(String taskId) + { + return new TaskStatusResponse( + taskId, + new TaskStatus(taskId, tasks.get(taskId), null, -1) + ); + } + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index f78dda0d56fe..802a34820e06 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -31,7 +31,6 @@ import com.google.common.io.Files; import com.google.inject.Binder; import com.google.inject.Module; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; @@ -43,6 +42,7 @@ import io.druid.guice.GuiceAnnotationIntrospector; import io.druid.guice.GuiceInjectableValues; import io.druid.guice.GuiceInjectors; +import io.druid.indexing.common.Counters; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.TestUtils; @@ -61,6 +61,7 @@ import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -216,7 +217,13 @@ public void deleteSegments(Set segments) }; final LocalTaskActionClientFactory tac = new LocalTaskActionClientFactory( TASK_STORAGE, - new TaskActionToolbox(TASK_LOCKBOX, mdc, newMockEmitter(), EasyMock.createMock(SupervisorManager.class)) + new TaskActionToolbox( + TASK_LOCKBOX, + mdc, + newMockEmitter(), + EasyMock.createMock(SupervisorManager.class), + new Counters() + ) ); SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class); EasyMock.replay(notifierFactory); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 5f9ade8ddfe6..2a63fe8c2242 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -44,6 +44,7 @@ import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; import io.druid.indexer.TaskState; +import io.druid.indexing.common.Counters; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -527,8 +528,16 @@ private TaskToolboxFactory setUpTaskToolboxFactory( Preconditions.checkNotNull(emitter); taskLockbox = new TaskLockbox(taskStorage); - tac = new LocalTaskActionClientFactory(taskStorage, new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock( - SupervisorManager.class))); + tac = new LocalTaskActionClientFactory( + taskStorage, + new TaskActionToolbox( + taskLockbox, + mdc, + emitter, + EasyMock.createMock(SupervisorManager.class), + new Counters() + ) + ); File tmpDir = temporaryFolder.newFolder(); taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null); diff --git a/server/src/main/java/io/druid/client/indexing/ClientQuery.java b/server/src/main/java/io/druid/client/indexing/ClientQuery.java new file mode 100644 index 000000000000..f692e3c2d597 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/ClientQuery.java @@ -0,0 +1,25 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client.indexing; + +public interface ClientQuery +{ + String getId(); +} diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index cfcf9fc431c6..9b7c98bf8915 100644 --- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.inject.Inject; -import io.druid.java.util.http.client.response.FullResponseHolder; import io.druid.discovery.DruidLeaderClient; import io.druid.indexer.TaskStatusPlus; import io.druid.java.util.common.DateTimes; @@ -32,6 +31,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.jackson.JacksonUtils; +import io.druid.java.util.http.client.response.FullResponseHolder; import io.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; @@ -77,17 +77,17 @@ public void mergeSegments(List segments) } } - runQuery(new ClientAppendQuery(dataSource, segments)); + runTask(new ClientAppendQuery(dataSource, segments)); } public void killSegments(String dataSource, Interval interval) { - runQuery(new ClientKillQuery(dataSource, interval)); + runTask(new ClientKillQuery(dataSource, interval)); } public void upgradeSegment(DataSegment dataSegment) { - runQuery(new ClientConversionQuery(dataSegment)); + runTask(new ClientConversionQuery(dataSegment)); } public String compactSegments( @@ -108,21 +108,21 @@ public String compactSegments( context = context == null ? new HashMap<>() : context; context.put("priority", compactionTaskPriority); - return runQuery(new ClientCompactQuery(dataSource, segments, tuningConfig, context)); + return runTask(new ClientCompactQuery(dataSource, segments, tuningConfig, context)); } - private String runQuery(Object queryObject) + public String runTask(Object taskObject) { try { final FullResponseHolder response = druidLeaderClient.go( druidLeaderClient.makeRequest( HttpMethod.POST, "/druid/indexer/v1/task" - ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(queryObject)) + ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject)) ); if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Failed to post query[%s]", queryObject); + throw new ISE("Failed to post task[%s]", taskObject); } final Map resultMap = jsonMapper.readValue( @@ -130,7 +130,40 @@ private String runQuery(Object queryObject) JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); final String taskId = (String) resultMap.get("task"); - return Preconditions.checkNotNull(taskId, "Null task id for query[%s]", queryObject); + return Preconditions.checkNotNull(taskId, "Null task id for task[%s]", taskObject); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public String killTask(String taskId) + { + try { + final FullResponseHolder response = druidLeaderClient.go( + druidLeaderClient.makeRequest( + HttpMethod.POST, + StringUtils.format("/druid/indexer/v1/task/%s/shutdown", taskId) + ) + ); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Failed to kill task[%s]", taskId); + } + + final Map resultMap = jsonMapper.readValue( + response.getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + final String killedTaskId = (String) resultMap.get("task"); + Preconditions.checkNotNull(killedTaskId, "Null task id returned for task[%s]", taskId); + Preconditions.checkState( + taskId.equals(killedTaskId), + "Requested to kill task[%s], but another task[%s] was killed!", + taskId, + killedTaskId + ); + return killedTaskId; } catch (Exception e) { throw Throwables.propagate(e); @@ -202,6 +235,30 @@ private List getTasks(String endpointSuffix) } } + @Nullable + public TaskStatusResponse getTaskStatus(String taskId) + { + try { + final FullResponseHolder responseHolder = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/task/%s/status", taskId)) + ); + + if (responseHolder.getStatus().equals(HttpResponseStatus.OK)) { + return jsonMapper.readValue( + responseHolder.getContent(), + new TypeReference() + { + } + ); + } else { + return null; + } + } + catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } + @Nullable public TaskStatusPlus getLastCompleteTask() { diff --git a/server/src/main/java/io/druid/client/indexing/TaskStatus.java b/server/src/main/java/io/druid/client/indexing/TaskStatus.java new file mode 100644 index 000000000000..8e255ca0d6a1 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/TaskStatus.java @@ -0,0 +1,114 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import io.druid.indexer.TaskReport; +import io.druid.indexer.TaskState; + +import javax.annotation.Nullable; + +/** + * Should be synced with io.druid.indexing.common.TaskStatus + */ +public class TaskStatus +{ + private final String id; + private final TaskState status; + private final TaskReport report; + private final long duration; + + @JsonCreator + public TaskStatus( + @JsonProperty("id") String id, + @JsonProperty("status") TaskState status, + @JsonProperty("report") @Nullable TaskReport report, + @JsonProperty("duration") long duration + ) + { + this.id = id; + this.status = status; + this.report = report; + this.duration = duration; + + // Check class invariants. + Preconditions.checkNotNull(id, "id"); + Preconditions.checkNotNull(status, "status"); + } + + @JsonProperty("id") + public String getId() + { + return id; + } + + @JsonProperty("status") + public TaskState getStatusCode() + { + return status; + } + + @JsonProperty("report") + public TaskReport getReport() + { + return report; + } + + @JsonProperty("duration") + public long getDuration() + { + return duration; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TaskStatus that = (TaskStatus) o; + return duration == that.duration && + java.util.Objects.equals(id, that.id) && + status == that.status && + java.util.Objects.equals(report, that.report); + } + + @Override + public int hashCode() + { + return java.util.Objects.hash(id, status, report, duration); + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("id", id) + .add("status", status) + .add("report", report) + .add("duration", duration) + .toString(); + } +} diff --git a/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java new file mode 100644 index 000000000000..07fed5532a17 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java @@ -0,0 +1,86 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * Should be synced with io.druid.indexing.overlord.http.TaskStatusResponse + */ +public class TaskStatusResponse +{ + private final String task; // Task ID, named "task" in the JSONification of this class. + private final TaskStatus status; + + @JsonCreator + public TaskStatusResponse( + @JsonProperty("task") final String task, + @JsonProperty("status") final TaskStatus status + ) + { + this.task = task; + this.status = status; + } + + @JsonProperty + public String getTask() + { + return task; + } + + @JsonProperty + public TaskStatus getStatus() + { + return status; + } + + @Override + public boolean equals(final Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final TaskStatusResponse that = (TaskStatusResponse) o; + return Objects.equals(task, that.task) && + Objects.equals(status, that.status); + } + + @Override + public int hashCode() + { + + return Objects.hash(task, status); + } + + @Override + public String toString() + { + return "TaskstatusResponse{" + + "task='" + task + '\'' + + ", status=" + status + + '}'; + } +} diff --git a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 17917d009e84..ceaaecacb5e9 100644 --- a/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -24,6 +24,7 @@ import org.joda.time.Interval; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Set; @@ -41,8 +42,10 @@ public interface IndexerMetadataStorageCoordinator * * @throws IOException */ - List getUsedSegmentsForInterval(String dataSource, Interval interval) - throws IOException; + default List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException + { + return getUsedSegmentsForIntervals(dataSource, Collections.singletonList(interval)); + } /** * Get all segments which may include any data in the interval and are flagged as used. diff --git a/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java b/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java index 0b9aa964602b..ad8e0f265177 100644 --- a/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java +++ b/server/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorSpec.java @@ -30,8 +30,14 @@ }) public interface SupervisorSpec { + /** + * Return an unique id of {@link Supervisor}. + */ String getId(); + /** + * Create a new {@link Supervisor} instance. + */ Supervisor createSupervisor(); List getDataSources(); diff --git a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 52067ab960c4..46ce0eb4ad37 100644 --- a/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/io/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -110,15 +110,6 @@ public void start() connector.createSegmentTable(); } - @Override - public List getUsedSegmentsForInterval( - final String dataSource, - final Interval interval - ) throws IOException - { - return getUsedSegmentsForIntervals(dataSource, ImmutableList.of(interval)); - } - @Override public List getUsedSegmentsForIntervals( final String dataSource, final List intervals diff --git a/server/src/main/java/io/druid/segment/indexing/IOConfig.java b/server/src/main/java/io/druid/segment/indexing/IOConfig.java index 94546855b784..04e947f6c812 100644 --- a/server/src/main/java/io/druid/segment/indexing/IOConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/IOConfig.java @@ -26,7 +26,7 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "realtime", value = RealtimeIOConfig.class), + @JsonSubTypes.Type(name = "realtime", value = RealtimeIOConfig.class) }) public interface IOConfig { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java index cf3ecea6fba2..bc83903b6297 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/HttpFirehoseFactory.java @@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.StringUtils; @@ -34,6 +37,7 @@ import java.net.URI; import java.net.URLConnection; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; @@ -146,4 +150,22 @@ protected Predicate getRetryCondition() { return e -> e instanceof IOException; } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + try { + return new HttpFirehoseFactory( + Collections.singletonList(split.get()), + getMaxCacheCapacityBytes(), + getMaxFetchCapacityBytes(), + getPrefetchTriggerBytes(), + getFetchTimeout(), + getMaxFetchRetry() + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java index 6db1e8c30941..4e598b258148 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -22,10 +22,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import io.druid.java.util.emitter.EmittingLogger; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; import io.druid.data.input.impl.AbstractTextFilesFirehoseFactory; import io.druid.data.input.impl.StringInputRowParser; import io.druid.java.util.common.CompressionUtils; +import io.druid.java.util.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; import org.apache.commons.io.filefilter.TrueFileFilter; import org.apache.commons.io.filefilter.WildcardFileFilter; @@ -99,4 +101,13 @@ protected InputStream wrapObjectStream(File object, InputStream stream) throws I { return object.getPath().endsWith(".gz") ? CompressionUtils.gzipInputStream(stream) : stream; } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + final File newFile = split.get(); + final File baseDir = newFile.getParentFile(); + final String filter = newFile.getName(); + return new LocalFirehoseFactory(baseDir, filter, parser); + } } diff --git a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java index 673bed3cf6a5..e32111bd6608 100644 --- a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; /** * An extendable linear shard spec containing the information of core partitions. This class contains two variables of @@ -109,4 +110,28 @@ public String toString() ", partitions=" + partitions + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (!(o instanceof NumberedShardSpec)) { + return false; + } + + final NumberedShardSpec that = (NumberedShardSpec) o; + if (partitionNum != that.partitionNum) { + return false; + } + return partitions == that.partitions; + } + + @Override + public int hashCode() + { + return Objects.hash(partitionNum, partitions); + } } diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index a205dfab01b3..f3107a2626e1 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -36,6 +36,7 @@ import io.airlift.airline.Option; import io.druid.client.cache.CacheConfig; import io.druid.client.coordinator.CoordinatorClient; +import io.druid.client.indexing.IndexingServiceClient; import io.druid.guice.Binders; import io.druid.guice.CacheModule; import io.druid.guice.DruidProcessingModule; @@ -168,6 +169,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); configureTaskActionClient(binder); + binder.bind(IndexingServiceClient.class).in(LazySingleton.class); binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index a50f51bf0204..bd963aab8e2f 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -138,7 +138,7 @@ public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) } } - private static class NoopDataSegmentPusher implements DataSegmentPusher + public static class NoopDataSegmentPusher implements DataSegmentPusher { @Override From fcaf1bf0bb379c606ec283dc2faa53cb453ba84a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 14 Mar 2018 20:51:36 -0700 Subject: [PATCH 02/47] fix build --- .../java/io/druid/indexing/common/task/TaskMonitorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java index 986372fd7028..314c7e21bdd9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java @@ -93,7 +93,7 @@ public void testRetry() throws InterruptedException, ExecutionException, Timeout } } - private class TestTask extends NoopTask + private static class TestTask extends NoopTask { private final int numMaxFails; From 6ce2db85d19928a2042522ecf3dc7cdba336df0d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 14 Mar 2018 22:14:29 -0700 Subject: [PATCH 03/47] fix ci --- .../main/java/io/druid/data/input/FiniteFirehoseFactory.java | 4 ++++ .../task/ParallelIndexSinglePhaseSupervisorTaskTest.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java index 012b8914d290..e9022d7becad 100644 --- a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java @@ -19,6 +19,7 @@ package io.druid.data.input; +import com.fasterxml.jackson.annotation.JsonIgnore; import io.druid.data.input.impl.InputRowParser; import java.io.IOException; @@ -36,6 +37,7 @@ public interface FiniteFirehoseFactory extends Fire /** * Returns true if the {@link FirehoseFactory} supports parallel batch indexing. */ + @JsonIgnore @Override default boolean isSplittable() { @@ -45,11 +47,13 @@ default boolean isSplittable() /** * Returns an iterator of {@link InputSplit}s. */ + @JsonIgnore Iterator> getSplits() throws IOException; /** * Returns number of splits returned by {@link #getSplits()}. */ + @JsonIgnore int getNumSplits() throws IOException; /** diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java index 72d7ce7bc1e6..884719642773 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java @@ -233,7 +233,7 @@ private ParallelIndexSinglePhaseSupervisorTask newTask(ParallelIndexSinglePhaseI ); } - private class TestParallelIndexSinglePhaseSupervisorTask extends ParallelIndexSinglePhaseSupervisorTask + private static class TestParallelIndexSinglePhaseSupervisorTask extends ParallelIndexSinglePhaseSupervisorTask { TestParallelIndexSinglePhaseSupervisorTask( From 0ace07ab8a4808d6b8d8bc96ac8052dd5a9e29f7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 19 Mar 2018 11:45:30 -0700 Subject: [PATCH 04/47] fix ingestion without intervals --- .../indexing/kafka/KafkaIndexTaskTest.java | 1 + .../SurrogateLockTryAcquireAction.java | 110 ++++++++++++++++++ .../indexing/common/actions/TaskAction.java | 1 + .../actions/TaskActionPreconditions.java | 7 +- .../common/actions/TaskActionToolbox.java | 11 +- .../ParallelIndexSinglePhaseIOConfig.java | 5 +- .../task/ParallelIndexSinglePhaseSubTask.java | 31 ++++- ...arallelIndexSinglePhaseSupervisorTask.java | 3 + .../indexing/common/task/TaskMonitor.java | 10 +- .../common/actions/TaskActionTestKit.java | 1 + ...penderatorDriverRealtimeIndexTaskTest.java | 1 + .../common/task/IngestionTestBase.java | 1 + ...lelIndexSinglePhaseSupervisorTaskTest.java | 30 ++++- .../common/task/RealtimeIndexTaskTest.java | 1 + .../IngestSegmentFirehoseFactoryTest.java | 1 + .../indexing/overlord/TaskLifecycleTest.java | 1 + 16 files changed, 205 insertions(+), 10 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index ef9c59e40da1..cac3fb997ab4 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1938,6 +1938,7 @@ private void makeToolboxFactory() throws IOException taskLockbox = new TaskLockbox(taskStorage); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( taskLockbox, + taskStorage, metadataStorageCoordinator, emitter, new SupervisorManager(null) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java new file mode 100644 index 000000000000..d50ff631ce08 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java @@ -0,0 +1,110 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Optional; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.overlord.LockResult; +import io.druid.java.util.common.ISE; +import org.joda.time.Interval; + +import javax.annotation.Nullable; + +public class SurrogateLockTryAcquireAction implements TaskAction +{ + private final TaskLockType type; + + private final Interval interval; + + private final String surrogateId; + + @JsonCreator + public SurrogateLockTryAcquireAction( + @JsonProperty("lockType") @Nullable TaskLockType type, + @JsonProperty("interval") Interval interval, + @JsonProperty("surrogateId") String surrogateId + ) + { + this.type = type; + this.interval = interval; + this.surrogateId = surrogateId; + } + + @JsonProperty("lockType") + public TaskLockType getType() + { + return type; + } + + @JsonProperty("interval") + public Interval getInterval() + { + return interval; + } + + @JsonProperty("surrogateId") + public String getSurrogateId() + { + return surrogateId; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public TaskLock perform( + Task task, TaskActionToolbox toolbox + ) + { + final Optional maybeSurrogateTask = toolbox.getTaskStorage().getTask(surrogateId); + if (maybeSurrogateTask.isPresent()) { + final LockResult result = toolbox.getTaskLockbox().tryLock(type, maybeSurrogateTask.get(), interval); + return result.isOk() ? result.getTaskLock() : null; + } else { + throw new ISE("Can't find surrogate task[%s]", surrogateId); + } + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return "LockTryAcquireAction{" + + "lockType=" + type + + ", interval=" + interval + + ", surrogateId=" + surrogateId + + '}'; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index b60c3622d799..8263450f87ee 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -30,6 +30,7 @@ @JsonSubTypes(value = { @JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class), @JsonSubTypes.Type(name = "lockTryAcquire", value = LockTryAcquireAction.class), + @JsonSubTypes.Type(name = "surrogateLockTryAcquire", value = SurrogateLockTryAcquireAction.class), @JsonSubTypes.Type(name = "lockList", value = LockListAction.class), @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java index 85cc74f6ec03..f1afc1fe3599 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionPreconditions.java @@ -42,7 +42,12 @@ static void checkLockCoversSegments( ) { if (!isLockCoversSegments(task, taskLockbox, segments)) { - throw new ISE("Segments not covered by locks for task: %s", task.getId()); + throw new ISE( + "Segments[%s] are not covered by locks[%s] for task[%s]", + segments, + taskLockbox.findLocksForTask(task), + task.getId() + ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java index 2bc0b771506c..7b06fedbb5c6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java @@ -21,14 +21,16 @@ import com.google.inject.Inject; import io.druid.indexing.common.Counters; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import io.druid.indexing.overlord.TaskLockbox; +import io.druid.indexing.overlord.TaskStorage; import io.druid.indexing.overlord.supervisor.SupervisorManager; +import io.druid.java.util.emitter.service.ServiceEmitter; public class TaskActionToolbox { private final TaskLockbox taskLockbox; + private final TaskStorage taskStorage; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private final ServiceEmitter emitter; private final SupervisorManager supervisorManager; @@ -37,6 +39,7 @@ public class TaskActionToolbox @Inject public TaskActionToolbox( TaskLockbox taskLockbox, + TaskStorage taskStorage, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, ServiceEmitter emitter, SupervisorManager supervisorManager, @@ -44,6 +47,7 @@ public TaskActionToolbox( ) { this.taskLockbox = taskLockbox; + this.taskStorage = taskStorage; this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator; this.emitter = emitter; this.supervisorManager = supervisorManager; @@ -55,6 +59,11 @@ public TaskLockbox getTaskLockbox() return taskLockbox; } + public TaskStorage getTaskStorage() + { + return taskStorage; + } + public IndexerMetadataStorageCoordinator getIndexerMetadataStorageCoordinator() { return indexerMetadataStorageCoordinator; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java index b2296a1fda78..28e122c62b0c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import io.druid.data.input.FirehoseFactory; import io.druid.indexing.common.task.IndexTask.IndexIOConfig; @@ -32,8 +33,8 @@ public class ParallelIndexSinglePhaseIOConfig extends IndexIOConfig { public ParallelIndexSinglePhaseIOConfig( - FirehoseFactory firehoseFactory, - @Nullable Boolean appendToExisting + @JsonProperty("firehose") FirehoseFactory firehoseFactory, + @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting ) { super(firehoseFactory, appendToExisting); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java index a09cfdfe62b3..812a17cc37ff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java @@ -34,8 +34,10 @@ import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.appenderator.CountingActionBasedSegmentAllocator; import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SurrogateLockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.IndexTask.IndexIOConfig; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; @@ -92,11 +94,14 @@ public class ParallelIndexSinglePhaseSubTask extends AbstractTask private static final String TYPE = "parallelIndexSinglePhaseSubIndex"; private final IndexTask.IndexIngestionSpec ingestionSchema; + private final String supervisorTaskId; + // TODO: add attempt @JsonCreator public ParallelIndexSinglePhaseSubTask( @JsonProperty("id") final String id, @JsonProperty("groupId") final String groupId, + @JsonProperty("supervisorTaskId") final String supervisorTaskId, @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("spec") final IndexTask.IndexIngestionSpec ingestionSchema, @JsonProperty("context") final Map context @@ -111,6 +116,7 @@ public ParallelIndexSinglePhaseSubTask( ); this.ingestionSchema = ingestionSchema; + this.supervisorTaskId = supervisorTaskId; if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) { throw new UnsupportedOperationException("Guaranteed rollup is not supported"); @@ -158,6 +164,12 @@ public IndexTask.IndexIngestionSpec getIngestionSchema() return ingestionSchema; } + @JsonProperty + public String getSupervisorTaskId() + { + return supervisorTaskId; + } + @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { @@ -184,7 +196,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception if (determineIntervals) { final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); intervals.addAll(shardSpecs.getIntervals()); - final Map locks = Tasks.tryAcquireExclusiveLocks(toolbox.getTaskActionClient(), intervals); + final Map locks = tryAcquireExclusiveSurrogateLocks(toolbox.getTaskActionClient(), intervals); versions = locks.entrySet().stream() .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); @@ -216,6 +228,23 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception return TaskStatus.success(getId(), new TaskReport(getId(), pushedSegments)); } + private Map tryAcquireExclusiveSurrogateLocks( + TaskActionClient client, + SortedSet intervals + ) + throws IOException + { + final Map lockMap = new HashMap<>(); + for (Interval interval : Tasks.computeCompactIntervals(intervals)) { + final TaskLock lock = Preconditions.checkNotNull( + client.submit(new SurrogateLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval, supervisorTaskId)), + "Cannot acquire a lock for interval[%s]", interval + ); + lockMap.put(interval, lock); + } + return lockMap; + } + private static boolean isGuaranteedRollup(IndexTask.IndexIOConfig ioConfig, IndexTask.IndexTuningConfig tuningConfig) { Preconditions.checkState( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java index 265ab22eb730..8a9551b03402 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java @@ -243,6 +243,8 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr if (state != TaskState.SUCCESS) { // if this fails, kill all sub tasks + // Note: this doesn't work when this task is killed by users. We need a way for gracefully shutting down tasks + // for resource cleanup. taskMonitor.killAll(); } } @@ -347,6 +349,7 @@ ParallelIndexSinglePhaseSubTask newTask(InputSplit split) return new ParallelIndexSinglePhaseSubTask( null, getGroupId(), + getId(), null, new IndexIngestionSpec( ingestionSchema.getDataSchema(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index cae69a2841ed..0b667df151ba 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -92,13 +92,13 @@ public void start(long taskStatusCheckingPeriod) log.warn("task[%s] failed!", taskId); if (monitorEntry.numRetry < maxRetry) { log.info( - "We still have chnaces[%d/%d]. Retrying task[%s]", + "We still have chnaces[%d/%d] to complete. Retrying task[%s]", monitorEntry.numRetry, maxRetry, taskId ); - retry(monitorEntry.task); monitorEntry.incrementNumRetry(); + retry(monitorEntry.task); } else { log.error( "task[%s] failed after [%d] retries", @@ -157,7 +157,10 @@ private void retry(Task task) public void killAll() { - taskFutureMap.keySet().forEach(indexingServiceClient::killTask); + taskFutureMap.keySet().forEach(taskId -> { + log.info("Request to kill subtask[%s]", taskId); + indexingServiceClient.killTask(taskId); + }); taskFutureMap.clear(); } @@ -168,6 +171,7 @@ public int getNumRunningTasks() private static class MonitorEntry { + // TODO: should change task id private final Task task; private final SettableFuture future; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java index 72780efdc73a..c63bf91794a7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/TaskActionTestKit.java @@ -110,6 +110,7 @@ public void before() ); taskActionToolbox = new TaskActionToolbox( taskLockbox, + taskStorage, metadataStorageCoordinator, new NoopServiceEmitter(), EasyMock.createMock(SupervisorManager.class), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index a607ebfe74bb..a817bd282cb4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1137,6 +1137,7 @@ public SegmentPublishResult announceHistoricalSegments( final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( taskLockbox, + taskStorage, mdc, emitter, EasyMock.createMock(SupervisorManager.class), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java index c3da4ae07630..23a40794b5b3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java @@ -88,6 +88,7 @@ public TaskActionToolbox createTaskActionToolbox() storageCoordinator.start(); return new TaskActionToolbox( lockbox, + taskStorage, storageCoordinator, new NoopServiceEmitter(), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java index 884719642773..d299e688bf7f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java @@ -49,6 +49,7 @@ import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; +import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -63,6 +64,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -133,6 +135,7 @@ public void teardown() public void testIsReady() throws Exception { final ParallelIndexSinglePhaseSupervisorTask task = newTask( + Intervals.of("2017/2018"), new ParallelIndexSinglePhaseIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false @@ -154,10 +157,29 @@ public void testIsReady() throws Exception } } + @Test + public void testWithoutInterval() throws Exception + { + final ParallelIndexSinglePhaseSupervisorTask task = newTask( + null, + new ParallelIndexSinglePhaseIOConfig( + new LocalFirehoseFactory(inputDir, "test_*", null), + false + ) + ); + actionClient = createActionClient(task); + toolbox = createTaskToolbox(task); + + prepareTaskForLocking(task); + Assert.assertTrue(task.isReady(actionClient)); + Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); + } + @Test() public void testRunInParallel() throws Exception { final ParallelIndexSinglePhaseSupervisorTask task = newTask( + Intervals.of("2017/2018"), new ParallelIndexSinglePhaseIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false @@ -175,6 +197,7 @@ public void testRunInParallel() throws Exception public void testRunInSequential() throws Exception { final ParallelIndexSinglePhaseSupervisorTask task = newTask( + Intervals.of("2017/2018"), new ParallelIndexSinglePhaseIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null) { @@ -195,7 +218,10 @@ public boolean isSplittable() Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); } - private ParallelIndexSinglePhaseSupervisorTask newTask(ParallelIndexSinglePhaseIOConfig ioConfig) + private ParallelIndexSinglePhaseSupervisorTask newTask( + Interval interval, + ParallelIndexSinglePhaseIOConfig ioConfig + ) { // set up ingestion spec final ParallelIndexSinglePhaseIngestionSpec singlePhaseIngestionSpec = new ParallelIndexSinglePhaseIngestionSpec( @@ -214,7 +240,7 @@ private ParallelIndexSinglePhaseSupervisorTask newTask(ParallelIndexSinglePhaseI new UniformGranularitySpec( Granularities.DAY, Granularities.MINUTE, - Arrays.asList(Intervals.of("2017/2018")) + interval == null ? null : Collections.singletonList(interval) ), null, getObjectMapper() diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index e491d15380a1..3ca4b2fb8c82 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -981,6 +981,7 @@ private TaskToolbox makeToolbox( taskLockbox.syncFromStorage(); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( taskLockbox, + taskStorage, mdc, emitter, EasyMock.createMock(SupervisorManager.class), diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 802a34820e06..bbebcef789f9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -219,6 +219,7 @@ public void deleteSegments(Set segments) TASK_STORAGE, new TaskActionToolbox( TASK_LOCKBOX, + TASK_STORAGE, mdc, newMockEmitter(), EasyMock.createMock(SupervisorManager.class), diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 2a63fe8c2242..95af97fea546 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -532,6 +532,7 @@ private TaskToolboxFactory setUpTaskToolboxFactory( taskStorage, new TaskActionToolbox( taskLockbox, + taskStorage, mdc, emitter, EasyMock.createMock(SupervisorManager.class), From 2e8dbe0756e8104282d07fea8f0201a14e093044 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 19 Mar 2018 18:35:46 -0700 Subject: [PATCH 05/47] fix retry --- .../java/io/druid/data/input/InputSplit.java | 8 + .../java/io/druid/indexer/TaskReport.java | 3 +- .../task/ParallelIndexSinglePhaseSubTask.java | 16 +- .../ParallelIndexSinglePhaseSubTaskSpec.java | 59 ++++++++ ...arallelIndexSinglePhaseSupervisorTask.java | 85 ++++++++--- .../indexing/common/task/SubTaskSpec.java | 65 ++++++++ .../indexing/common/task/TaskMonitor.java | 143 ++++++++++++++---- ...lelIndexSinglePhaseSupervisorTaskTest.java | 20 ++- .../indexing/common/task/TaskMonitorTest.java | 74 +++++++-- 9 files changed, 393 insertions(+), 80 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java diff --git a/api/src/main/java/io/druid/data/input/InputSplit.java b/api/src/main/java/io/druid/data/input/InputSplit.java index 9e775c796391..70d7223ba4df 100644 --- a/api/src/main/java/io/druid/data/input/InputSplit.java +++ b/api/src/main/java/io/druid/data/input/InputSplit.java @@ -35,4 +35,12 @@ public T get() { return split; } + + @Override + public String toString() + { + return "InputSplit{" + + "split=" + split + + "}"; + } } diff --git a/api/src/main/java/io/druid/indexer/TaskReport.java b/api/src/main/java/io/druid/indexer/TaskReport.java index 74d64e658547..53dea93d97a7 100644 --- a/api/src/main/java/io/druid/indexer/TaskReport.java +++ b/api/src/main/java/io/druid/indexer/TaskReport.java @@ -35,7 +35,8 @@ public class TaskReport @JsonCreator public TaskReport( @JsonProperty("taskId") String taskId, - @JsonProperty("payload") Object payload) + @JsonProperty("payload") Object payload + ) { this.taskId = taskId; this.payload = payload; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java index 812a17cc37ff..d4968c4f62a5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java @@ -93,28 +93,29 @@ public class ParallelIndexSinglePhaseSubTask extends AbstractTask private static final Logger log = new Logger(ParallelIndexSinglePhaseSubTask.class); private static final String TYPE = "parallelIndexSinglePhaseSubIndex"; + private final int numAttempts; private final IndexTask.IndexIngestionSpec ingestionSchema; private final String supervisorTaskId; - // TODO: add attempt @JsonCreator public ParallelIndexSinglePhaseSubTask( - @JsonProperty("id") final String id, @JsonProperty("groupId") final String groupId, - @JsonProperty("supervisorTaskId") final String supervisorTaskId, @JsonProperty("resource") final TaskResource taskResource, + @JsonProperty("supervisorTaskId") final String supervisorTaskId, + @JsonProperty("numAttempts") final int numAttempts, // zero-based counting @JsonProperty("spec") final IndexTask.IndexIngestionSpec ingestionSchema, @JsonProperty("context") final Map context ) { super( - getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + getOrMakeId(null, TYPE, ingestionSchema.getDataSchema().getDataSource()), groupId, taskResource, ingestionSchema.getDataSchema().getDataSource(), context ); + this.numAttempts = numAttempts; this.ingestionSchema = ingestionSchema; this.supervisorTaskId = supervisorTaskId; @@ -147,7 +148,6 @@ public boolean isReady(TaskActionClient taskActionClient) private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) { - // TODO: check when intervals are not specified try { Tasks.tryAcquireExclusiveLocks(actionClient, intervals); return true; @@ -158,6 +158,12 @@ private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet +{ + private final IndexIngestionSpec ingestionSpec; + + ParallelIndexSinglePhaseSubTaskSpec( + String id, + String groupId, + String supervisorTaskId, + IndexIngestionSpec indexIngestionSpec, + Map context + ) + { + super(id, groupId, supervisorTaskId, context); + this.ingestionSpec = indexIngestionSpec; + } + + public IndexIngestionSpec getIngestionSpec() + { + return ingestionSpec; + } + + @Override + public ParallelIndexSinglePhaseSubTask newSubTask(int numAttempts) + { + return new ParallelIndexSinglePhaseSubTask( + getGroupId(), + null, + getSupervisorTaskId(), + numAttempts, + getIngestionSpec(), + getContext() + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java index 8a9551b03402..d6c7abf48076 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java @@ -26,6 +26,7 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -43,8 +44,10 @@ import io.druid.indexing.common.task.IndexTask.IndexIOConfig; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.RE; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; @@ -83,10 +86,11 @@ public class ParallelIndexSinglePhaseSupervisorTask extends AbstractTask private final int maxNumTasks; private final TaskMonitor taskMonitor; - private final BlockingQueue taskCompleteEvents = new LinkedBlockingDeque<>(); + private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); private final List segments = new ArrayList<>(); private volatile boolean stopped; + private int nextSpecId = 0; @JsonCreator public ParallelIndexSinglePhaseSupervisorTask( @@ -176,7 +180,7 @@ public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Excep private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) throws Exception { - final Iterator subTaskIterator = subTaskIterator(); + final Iterator subTaskIterator = subTaskSpecIterator(); final int numTotalTasks = baseFirehoseFactory.getNumSplits(); final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckingPeriodMs(); TaskState state = TaskState.FAILED; @@ -194,17 +198,17 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr } while (!stopped && !Thread.currentThread().isInterrupted()) { - final TaskStatus taskCompleteEvent = taskCompleteEvents.poll(taskStatusCheckingPeriod, TimeUnit.MILLISECONDS); + final SubTaskCompleteEvent taskCompleteEvent = taskCompleteEvents.poll( + taskStatusCheckingPeriod, + TimeUnit.MILLISECONDS + ); if (taskCompleteEvent != null) { - final TaskState completeState = taskCompleteEvent.getStatusCode(); - if (completeState == null) { - throw new ISE("Complete state of task[%s] is null", taskCompleteEvent.getId()); - } + final TaskState completeState = taskCompleteEvent.getLastState(); switch (completeState) { case SUCCESS: numCompleteTasks++; - segments.addAll((Collection) taskCompleteEvent.getReport().getPayload()); + segments.addAll((Collection) taskCompleteEvent.getLastStatus().getReport().getPayload()); log.info("[%d/%d] tasks succeeded", numCompleteTasks, numTotalTasks); if (!subTaskIterator.hasNext()) { if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) { @@ -217,7 +221,11 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr state = TaskState.SUCCESS; } else { // Failed - throw new ISE("Expected to complete [%d] tasks, but we got [%d]", numTotalTasks, numCompleteTasks); + throw new ISE( + "Expected to complete [%d] tasks, but we got [%d] tasks", + numTotalTasks, + numCompleteTasks + ); } } } else if (taskMonitor.getNumRunningTasks() < maxNumTasks) { @@ -228,10 +236,20 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr // TaskMonitor already tried everything it can do for failed tasks. We failed. state = TaskState.FAILED; stopped = true; - log.error("Failed because of the failed sub task[%s]", taskCompleteEvent.getId()); + final TaskStatus lastStatus = taskCompleteEvent.getLastStatus(); + if (lastStatus != null) { + log.error("Failed because of the failed sub task[%s]", lastStatus.getId()); + } else { + final ParallelIndexSinglePhaseSubTaskSpec spec = + (ParallelIndexSinglePhaseSubTaskSpec) taskCompleteEvent.getSpec(); + log.error( + "Failed to run sub tasks for inputSplits[%s]", + getSplitsIfSplittable(spec.getIngestionSpec().getIOConfig().getFirehoseFactory()) + ); + } break; default: - throw new ISE("Complete task[%s] is in an invalid state[%s]", taskCompleteEvent.getId(), completeState); + throw new ISE("spec[%s] is in an invalid state[%s]", taskCompleteEvent.getSpec().getId(), completeState); } } } @@ -315,42 +333,53 @@ private void publish(TaskToolbox toolbox) throws IOException } } - private void submitNewTask(ParallelIndexSinglePhaseSubTask task) + private void submitNewTask(ParallelIndexSinglePhaseSubTaskSpec spec) { - final ListenableFuture future = taskMonitor.submit(task); + final ListenableFuture> future = taskMonitor.submit(spec); Futures.addCallback( future, - new FutureCallback() + new FutureCallback>() { @Override - public void onSuccess(TaskStatus taskStatus) + public void onSuccess(SubTaskCompleteEvent completeEvent) { - taskCompleteEvents.offer(taskStatus); + // this callback is called if a task completed wheter it succeeded or not. + taskCompleteEvents.offer(completeEvent); } @Override public void onFailure(Throwable t) { - taskCompleteEvents.offer(new TaskStatus(task.getId(), TaskState.FAILED, null, -1)); + // this callback is called only when there were some problems in TaskMonitor. + try { + log.error( + t, + "Error while running a task for inputSplits[%s]", + getSplitsIfSplittable(spec.getIngestionSpec().getIOConfig().getFirehoseFactory()) + ); + } + catch (IOException e) { + t.addSuppressed(new RE(e, "Error while getting splits for error logging")); + } + taskCompleteEvents.offer(new SubTaskCompleteEvent<>(spec, TaskState.FAILED, null)); } } ); } @VisibleForTesting - Iterator subTaskIterator() throws IOException + Iterator subTaskSpecIterator() throws IOException { - return Iterators.transform(baseFirehoseFactory.getSplits(), split -> newTask((InputSplit) split)); + return Iterators.transform(baseFirehoseFactory.getSplits(), split -> newTaskSpec((InputSplit) split)); } @VisibleForTesting - ParallelIndexSinglePhaseSubTask newTask(InputSplit split) + ParallelIndexSinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { - return new ParallelIndexSinglePhaseSubTask( - null, + return new ParallelIndexSinglePhaseSubTaskSpec( + getId() + "_" + nextSpecId++, getGroupId(), getId(), - null, new IndexIngestionSpec( ingestionSchema.getDataSchema(), new IndexIOConfig( @@ -362,4 +391,14 @@ ParallelIndexSinglePhaseSubTask newTask(InputSplit split) getContext() ); } + + private static List getSplitsIfSplittable(FirehoseFactory firehoseFactory) throws IOException + { + if (firehoseFactory instanceof FiniteFirehoseFactory) { + final FiniteFirehoseFactory finiteFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; + return Lists.newArrayList(finiteFirehoseFactory.getSplits()); + } else { + throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName()); + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java new file mode 100644 index 000000000000..2472d25c81a3 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java @@ -0,0 +1,65 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import java.util.Map; + +public abstract class SubTaskSpec +{ + private final String id; + private final String groupId; + private final String supervisorTaskId; + private final Map context; + + public SubTaskSpec( + String id, + String groupId, + String supervisorTaskId, + Map context + ) + { + this.id = id; + this.groupId = groupId; + this.supervisorTaskId = supervisorTaskId; + this.context = context; + } + + public String getId() + { + return id; + } + + public String getGroupId() + { + return groupId; + } + + public String getSupervisorTaskId() + { + return supervisorTaskId; + } + + public Map getContext() + { + return context; + } + + public abstract T newSubTask(int numAttempts); +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index 0b667df151ba..a85cb46bcf7b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -26,11 +26,15 @@ import io.druid.client.indexing.IndexingServiceClient; import io.druid.client.indexing.TaskStatus; import io.druid.client.indexing.TaskStatusResponse; +import io.druid.indexer.TaskState; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -42,13 +46,14 @@ * Responsible for submitting tasks, monitoring task statuses, resubmitting failed tasks, and returning the final task * status. */ -public class TaskMonitor +public class TaskMonitor { private static final Logger log = new Logger(TaskMonitor.class); private final ScheduledExecutorService taskStatusChecker = Execs.scheduledSingleThreaded(("task-monitor-%d")); - private final ConcurrentMap taskFutureMap = new ConcurrentHashMap<>(); + // taskId -> monitorEntry + private final ConcurrentMap runningTasks = new ConcurrentHashMap<>(); // overlord client private final AtomicInteger numRunningTasks = new AtomicInteger(); @@ -73,7 +78,7 @@ public void start(long taskStatusCheckingPeriod) taskStatusChecker.scheduleAtFixedRate( () -> { try { - final Iterator> iterator = taskFutureMap.entrySet().iterator(); + final Iterator> iterator = runningTasks.entrySet().iterator(); while (iterator.hasNext()) { final Entry entry = iterator.next(); final String taskId = entry.getKey(); @@ -90,20 +95,19 @@ public void start(long taskStatusCheckingPeriod) case FAILED: numRunningTasks.decrementAndGet(); log.warn("task[%s] failed!", taskId); - if (monitorEntry.numRetry < maxRetry) { + if (monitorEntry.numTries() < maxRetry) { log.info( "We still have chnaces[%d/%d] to complete. Retrying task[%s]", - monitorEntry.numRetry, + monitorEntry.numTries(), maxRetry, taskId ); - monitorEntry.incrementNumRetry(); - retry(monitorEntry.task); + retry(monitorEntry, taskStatus); } else { log.error( - "task[%s] failed after [%d] retries", + "task[%s] failed after [%d] tries", taskId, - monitorEntry.numRetry + monitorEntry.numTries() ); iterator.remove(); monitorEntry.setLastStatus(taskStatus); @@ -132,36 +136,41 @@ public void stop() log.info("Stopped taskMonitor"); } - public ListenableFuture submit(Task task) + public ListenableFuture> submit(SubTaskSpec spec) { if (!running) { return Futures.immediateFailedFuture(new ISE("TaskMonitore is not running")); } + final T task = spec.newSubTask(0); log.info("Submitting a new task[%s]", task.getId()); - final String taskId = indexingServiceClient.runTask(task); - + indexingServiceClient.runTask(task); numRunningTasks.incrementAndGet(); - final SettableFuture taskFuture = SettableFuture.create(); - taskFutureMap.put(taskId, new MonitorEntry(task, taskFuture)); + final SettableFuture> taskFuture = SettableFuture.create(); + runningTasks.put(task.getId(), new MonitorEntry(spec, task, taskFuture)); + return taskFuture; } - private void retry(Task task) + private void retry(MonitorEntry monitorEntry, TaskStatus lastFailedTaskStatus) { if (running) { + final SubTaskSpec spec = monitorEntry.spec; + final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1); indexingServiceClient.runTask(task); numRunningTasks.incrementAndGet(); + + runningTasks.put(task.getId(), monitorEntry.withNewRunningTask(task, lastFailedTaskStatus)); } } public void killAll() { - taskFutureMap.keySet().forEach(taskId -> { + runningTasks.keySet().forEach(taskId -> { log.info("Request to kill subtask[%s]", taskId); indexingServiceClient.killTask(taskId); }); - taskFutureMap.clear(); + runningTasks.clear(); } public int getNumRunningTasks() @@ -169,28 +178,104 @@ public int getNumRunningTasks() return numRunningTasks.intValue(); } - private static class MonitorEntry + private class MonitorEntry { - // TODO: should change task id - private final Task task; - private final SettableFuture future; + private final SubTaskSpec spec; + private final T runningTask; + private final List taskHistory; + private final SettableFuture> completeEventFuture; + + MonitorEntry( + SubTaskSpec spec, + T runningTask, + SettableFuture> completeEventFuture + ) + { + this(spec, runningTask, new ArrayList<>(), completeEventFuture); + } + + private MonitorEntry( + SubTaskSpec spec, + T runningTask, + List taskHistory, + SettableFuture> completeEventFuture + ) + { + this.spec = spec; + this.runningTask = runningTask; + this.taskHistory = taskHistory; + this.completeEventFuture = completeEventFuture; + } - private int numRetry; + MonitorEntry withNewRunningTask(T newTask, TaskStatus statusOfLastTask) + { + taskHistory.add(statusOfLastTask); + return new MonitorEntry( + spec, + newTask, + taskHistory, + completeEventFuture + ); + } + + int numTries() + { + return taskHistory.size() + 1; // count runningTask. this is valid only until setLastStatus() is called + } + + void setLastStatus(TaskStatus lastStatus) + { + if (!runningTask.getId().equals(lastStatus.getId())) { + throw new ISE( + "Task id[%s] of lastStatus is different from the running task[%s]", + lastStatus.getId(), + runningTask.getId() + ); + } + + taskHistory.add(lastStatus); + completeEventFuture.set(new SubTaskCompleteEvent<>(spec, lastStatus.getStatusCode(), taskHistory)); + } + } + + static class SubTaskCompleteEvent + { + private final SubTaskSpec spec; + private final TaskState lastState; + @Nullable + private final List attemptHistory; + + SubTaskCompleteEvent( + SubTaskSpec spec, + TaskState lastState, + @Nullable List attemptHistory + ) + { + this.spec = Preconditions.checkNotNull(spec, "spec"); + this.lastState = Preconditions.checkNotNull(lastState, "lastState"); + this.attemptHistory = attemptHistory; + } + + SubTaskSpec getSpec() + { + return spec; + } - MonitorEntry(Task task, SettableFuture future) + TaskState getLastState() { - this.task = task; - this.future = future; + return lastState; } - void setLastStatus(TaskStatus taskStatus) + @Nullable + List getAttemptHistory() { - future.set(taskStatus); + return attemptHistory; } - void incrementNumRetry() + @Nullable + TaskStatus getLastStatus() { - numRetry++; + return attemptHistory == null ? null : attemptHistory.get(attemptHistory.size() - 1); } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java index d299e688bf7f..73eab386ce65 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java @@ -147,10 +147,18 @@ public void testIsReady() throws Exception prepareTaskForLocking(task); Assert.assertTrue(task.isReady(actionClient)); - final Iterator subTaskIterator = task.subTaskIterator(); - - while (subTaskIterator.hasNext()) { - final ParallelIndexSinglePhaseSubTask subTask = subTaskIterator.next(); + final Iterator subTaskSpecIterator = task.subTaskSpecIterator(); + + while (subTaskSpecIterator.hasNext()) { + final ParallelIndexSinglePhaseSubTaskSpec spec = subTaskSpecIterator.next(); + final ParallelIndexSinglePhaseSubTask subTask = new ParallelIndexSinglePhaseSubTask( + spec.getGroupId(), + null, + spec.getSupervisorTaskId(), + 0, + spec.getIngestionSpec(), + spec.getContext() + ); final TaskActionClient subTaskActionClient = createActionClient(subTask); prepareTaskForLocking(subTask); Assert.assertTrue(subTask.isReady(subTaskActionClient)); @@ -274,7 +282,7 @@ private static class TestParallelIndexSinglePhaseSupervisorTask extends Parallel } @Override - Iterator subTaskIterator() throws IOException + Iterator subTaskSpecIterator() throws IOException { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() @@ -287,7 +295,7 @@ Iterator subTaskIterator() throws IOException catch (InterruptedException e) { throw new RuntimeException(e); } - return newTask((InputSplit) split); + return newTaskSpec((InputSplit) split); }); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java index 314c7e21bdd9..778221e8f6aa 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java @@ -25,6 +25,7 @@ import io.druid.client.indexing.TaskStatusResponse; import io.druid.indexer.TaskState; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; import io.druid.java.util.common.concurrent.Execs; import org.junit.After; import org.junit.Assert; @@ -32,6 +33,7 @@ import org.junit.Test; import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; @@ -45,7 +47,7 @@ public class TaskMonitorTest { private final ExecutorService taskRunner = Execs.multiThreaded(5, "task-monitor-test-%d"); private final ConcurrentMap tasks = new ConcurrentHashMap<>(); - private final TaskMonitor monitor = new TaskMonitor(new TestIndexingServiceClient(), 3); + private final TaskMonitor monitor = new TaskMonitor<>(new TestIndexingServiceClient(), 3); @Before public void setup() @@ -64,52 +66,92 @@ public void teardown() @Test public void testBasic() throws InterruptedException, ExecutionException, TimeoutException { - final List> futures = IntStream + final List>> futures = IntStream .range(0, 10) - .mapToObj(i -> monitor.submit(new TestTask("id" + i, 100, 0))) + .mapToObj(i -> monitor.submit(new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, 100L, 0))) .collect(Collectors.toList()); for (int i = 0; i < futures.size(); i++) { // # of threads of taskRunner is 5, so the expected max timeout is 2 sec. We additionally wait three more seconds // here to make sure the test passes. - final TaskStatus result = futures.get(i).get(1, TimeUnit.SECONDS); - Assert.assertEquals("id" + i, result.getId()); - Assert.assertEquals(TaskState.SUCCESS, result.getStatusCode()); + final SubTaskCompleteEvent result = futures.get(i).get(1, TimeUnit.SECONDS); + Assert.assertEquals("supervisorId", result.getSpec().getSupervisorTaskId()); + Assert.assertEquals("specId" + i, result.getSpec().getId()); + Assert.assertNotNull(result.getLastStatus()); + Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.getLastState()); } } @Test public void testRetry() throws InterruptedException, ExecutionException, TimeoutException { - final List> futures = IntStream + final List>> futures = IntStream .range(0, 10) - .mapToObj(i -> monitor.submit(new TestTask("id" + i, 100, 2))) + .mapToObj(i -> monitor.submit(new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, 100L, 2))) .collect(Collectors.toList()); for (int i = 0; i < futures.size(); i++) { // # of threads of taskRunner is 5, and each task is expected to be run 3 times (with 2 retries), so the expected // max timeout is 6 sec. We additionally wait 4 more seconds here to make sure the test passes. - final TaskStatus result = futures.get(i).get(2, TimeUnit.SECONDS); - Assert.assertEquals("id" + i, result.getId()); - Assert.assertEquals(TaskState.SUCCESS, result.getStatusCode()); + final SubTaskCompleteEvent result = futures.get(i).get(2, TimeUnit.SECONDS); + Assert.assertEquals("supervisorId", result.getSpec().getSupervisorTaskId()); + Assert.assertEquals("specId" + i, result.getSpec().getId()); + + Assert.assertNotNull(result.getLastStatus()); + Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.getLastState()); + + final List attemptHistory = result.getAttemptHistory(); + Assert.assertNotNull(attemptHistory); + Assert.assertEquals(3, attemptHistory.size()); + Assert.assertEquals(TaskState.FAILED, attemptHistory.get(0).getStatusCode()); + Assert.assertEquals(TaskState.FAILED, attemptHistory.get(1).getStatusCode()); } } - private static class TestTask extends NoopTask + private static class TestTaskSpec extends SubTaskSpec { + private final long runTime; private final int numMaxFails; private int numFails; - public TestTask(String id, long runTime, int numMaxFails) + public TestTaskSpec( + String id, + String groupId, + String supervisorTaskId, + Map context, + long runTime, + int numMaxFails + ) { - super(id, "testDataSource", runTime, 0, null, null, null); + super(id, groupId, supervisorTaskId, context); + this.runTime = runTime; this.numMaxFails = numMaxFails; } + @Override + public TestTask newSubTask(int numAttempts) + { + return new TestTask(getId(), numAttempts, runTime, numFails++ < numMaxFails); + } + } + + private static class TestTask extends NoopTask + { + private final int numAttempts; + private final boolean shouldFail; + + TestTask(String id, int numAttempts, long runTime, boolean shouldFail) + { + super(id, "testDataSource", runTime, 0, null, null, null); + this.numAttempts = numAttempts; + this.shouldFail = shouldFail; + } + @Override public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Exception { - if (numFails < numMaxFails) { - numFails++; + if (shouldFail) { Thread.sleep(getRunTime()); return io.druid.indexing.common.TaskStatus.failure(getId()); } else { From 67ca45c1447eb20c49ff421f79822f4c1dfe74c2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 20 Mar 2018 16:36:19 -0700 Subject: [PATCH 06/47] fix retry --- .../actions/SurrogateLockListAction.java | 78 +++++++++++++++++++ .../SurrogateLockTryAcquireAction.java | 6 +- .../indexing/common/actions/TaskAction.java | 1 + .../task/ParallelIndexSinglePhaseSubTask.java | 22 ++---- .../ParallelIndexSinglePhaseSubTaskSpec.java | 1 + ...arallelIndexSinglePhaseSupervisorTask.java | 33 ++++---- .../ParallelIndexSinglePhaseTuningConfig.java | 35 ++++----- .../indexing/common/task/TaskMonitor.java | 22 ++++-- ...lelIndexSinglePhaseSupervisorTaskTest.java | 17 +++- 9 files changed, 154 insertions(+), 61 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockListAction.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockListAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockListAction.java new file mode 100644 index 000000000000..30280ded262c --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockListAction.java @@ -0,0 +1,78 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Optional; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.task.Task; +import io.druid.java.util.common.ISE; + +import java.util.List; + +public class SurrogateLockListAction implements TaskAction> +{ + private final String surrogateId; + + @JsonCreator + public SurrogateLockListAction( + @JsonProperty("surrogateId") String surrogateId + ) + { + this.surrogateId = surrogateId; + } + + @JsonProperty + public String getSurrogateId() + { + return surrogateId; + } + + @Override + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public List perform(Task task, TaskActionToolbox toolbox) + { + final Optional maybeSurrogateTask = toolbox.getTaskStorage().getTask(surrogateId); + if (maybeSurrogateTask.isPresent()) { + return toolbox.getTaskLockbox().findLocksForTask(maybeSurrogateTask.get()); + } else { + throw new ISE("Can't find surrogate task[%s]", surrogateId); + } + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return "LockListAction{}"; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java index d50ff631ce08..316466825d4c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java @@ -30,8 +30,6 @@ import io.druid.java.util.common.ISE; import org.joda.time.Interval; -import javax.annotation.Nullable; - public class SurrogateLockTryAcquireAction implements TaskAction { private final TaskLockType type; @@ -42,7 +40,7 @@ public class SurrogateLockTryAcquireAction implements TaskAction @JsonCreator public SurrogateLockTryAcquireAction( - @JsonProperty("lockType") @Nullable TaskLockType type, + @JsonProperty("lockType") TaskLockType type, @JsonProperty("interval") Interval interval, @JsonProperty("surrogateId") String surrogateId ) @@ -101,7 +99,7 @@ public boolean isAudited() @Override public String toString() { - return "LockTryAcquireAction{" + + return "SurrogateLockTryAcquireAction{" + "lockType=" + type + ", interval=" + interval + ", surrogateId=" + surrogateId + diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index d10bfe01e615..368909b851dc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -30,6 +30,7 @@ @JsonSubTypes.Type(name = "lockTryAcquire", value = LockTryAcquireAction.class), @JsonSubTypes.Type(name = "surrogateLockTryAcquire", value = SurrogateLockTryAcquireAction.class), @JsonSubTypes.Type(name = "lockList", value = LockListAction.class), + @JsonSubTypes.Type(name = "surrogateLockList", value = SurrogateLockListAction.class), @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java index d4968c4f62a5..f2a2f7ba07fd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java @@ -37,6 +37,7 @@ import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SurrogateLockListAction; import io.druid.indexing.common.actions.SurrogateLockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.IndexTask.IndexIOConfig; @@ -69,6 +70,7 @@ import org.codehaus.plexus.util.FileUtils; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -99,6 +101,8 @@ public class ParallelIndexSinglePhaseSubTask extends AbstractTask @JsonCreator public ParallelIndexSinglePhaseSubTask( + // id shouldn't be null except when this task is created by ParallelIndexSinglePhaseSupervisorTask + @JsonProperty("id") @Nullable final String id, @JsonProperty("groupId") final String groupId, @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("supervisorTaskId") final String supervisorTaskId, @@ -108,7 +112,7 @@ public ParallelIndexSinglePhaseSubTask( ) { super( - getOrMakeId(null, TYPE, ingestionSchema.getDataSchema().getDataSource()), + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), groupId, taskResource, ingestionSchema.getDataSchema().getDataSource(), @@ -119,7 +123,7 @@ public ParallelIndexSinglePhaseSubTask( this.ingestionSchema = ingestionSchema; this.supervisorTaskId = supervisorTaskId; - if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) { + if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { throw new UnsupportedOperationException("Guaranteed rollup is not supported"); } } @@ -149,7 +153,7 @@ public boolean isReady(TaskActionClient taskActionClient) private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) { try { - Tasks.tryAcquireExclusiveLocks(actionClient, intervals); + tryAcquireExclusiveSurrogateLocks(actionClient, intervals); return true; } catch (Exception e) { @@ -216,7 +220,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ) ); } else { - versions = getTaskLocks(toolbox.getTaskActionClient()) + versions = toolbox.getTaskActionClient().submit(new SurrogateLockListAction(supervisorTaskId)) .stream() .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); dataSchema = ingestionSchema.getDataSchema(); @@ -251,16 +255,6 @@ private Map tryAcquireExclusiveSurrogateLocks( return lockMap; } - private static boolean isGuaranteedRollup(IndexTask.IndexIOConfig ioConfig, IndexTask.IndexTuningConfig tuningConfig) - { - Preconditions.checkState( - !(tuningConfig.isForceGuaranteedRollup() && - (tuningConfig.isForceExtendableShardSpecs() || ioConfig.isAppendToExisting())), - "Perfect rollup cannot be guaranteed with extendable shardSpecs" - ); - return tuningConfig.isForceGuaranteedRollup(); - } - /** * Determines intervals and shardSpecs for input data. This method first checks that it must determine intervals and * shardSpecs by itself. Intervals must be determined if they are not specified in {@link GranularitySpec}. diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java index ca31c8bc716f..6a10deb2cf4e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java @@ -48,6 +48,7 @@ public IndexIngestionSpec getIngestionSpec() public ParallelIndexSinglePhaseSubTask newSubTask(int numAttempts) { return new ParallelIndexSinglePhaseSubTask( + null, getGroupId(), null, getSupervisorTaskId(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java index d6c7abf48076..4293d870bbb8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTask.java @@ -82,9 +82,9 @@ public class ParallelIndexSinglePhaseSupervisorTask extends AbstractTask private static final String TYPE = "parallelIndexSinglePhase"; private final ParallelIndexSinglePhaseIngestionSpec ingestionSchema; - private final FiniteFirehoseFactory baseFirehoseFactory; + private final FiniteFirehoseFactory baseFirehoseFactory; private final int maxNumTasks; - private final TaskMonitor taskMonitor; + private final TaskMonitor taskMonitor; private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); private final List segments = new ArrayList<>(); @@ -118,7 +118,7 @@ public ParallelIndexSinglePhaseSupervisorTask( this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumBatchTasks(); - this.taskMonitor = new TaskMonitor(indexingServiceClient, ingestionSchema.getTuningConfig().getMaxRetry()); + this.taskMonitor = new TaskMonitor<>(indexingServiceClient, ingestionSchema.getTuningConfig().getMaxRetry()); } @Override @@ -180,9 +180,9 @@ public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Excep private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) throws Exception { - final Iterator subTaskIterator = subTaskSpecIterator(); + final Iterator subTaskSpecIterator = subTaskSpecIterator(); final int numTotalTasks = baseFirehoseFactory.getNumSplits(); - final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckingPeriodMs(); + final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs(); TaskState state = TaskState.FAILED; log.info("Total number of tasks is [%d]", numTotalTasks); @@ -193,10 +193,11 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr try { log.info("Submitting initial tasks"); // Submit initial tasks - while (subTaskIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) { - submitNewTask(subTaskIterator.next()); + while (subTaskSpecIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) { + submitNewTask(subTaskSpecIterator.next()); } + log.info("Waiting for subTasks to be completed"); while (!stopped && !Thread.currentThread().isInterrupted()) { final SubTaskCompleteEvent taskCompleteEvent = taskCompleteEvents.poll( taskStatusCheckingPeriod, @@ -210,7 +211,7 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr numCompleteTasks++; segments.addAll((Collection) taskCompleteEvent.getLastStatus().getReport().getPayload()); log.info("[%d/%d] tasks succeeded", numCompleteTasks, numTotalTasks); - if (!subTaskIterator.hasNext()) { + if (!subTaskSpecIterator.hasNext()) { if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) { stopped = true; if (numCompleteTasks == numTotalTasks) { @@ -229,7 +230,7 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr } } } else if (taskMonitor.getNumRunningTasks() < maxNumTasks) { - submitNewTask(subTaskIterator.next()); + submitNewTask(subTaskSpecIterator.next()); } break; case FAILED: @@ -255,11 +256,17 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr } } finally { + log.info("Cleaning up resources"); // Cleanup resources taskCompleteEvents.clear(); taskMonitor.stop(); if (state != TaskState.SUCCESS) { + log.info( + "This task is finished with [%s] state. Killing [%d] remaining subtasks.", + state, + taskMonitor.getNumRunningTasks() + ); // if this fails, kill all sub tasks // Note: this doesn't work when this task is killed by users. We need a way for gracefully shutting down tasks // for resource cleanup. @@ -298,7 +305,7 @@ private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexSingleP tuningConfig.getMaxPendingPersists(), true, tuningConfig.isForceExtendableShardSpecs(), - tuningConfig.isForceGuaranteedRollup(), + false, tuningConfig.isReportParseExceptions(), null, tuningConfig.getPushTimeout(), @@ -370,11 +377,11 @@ public void onFailure(Throwable t) @VisibleForTesting Iterator subTaskSpecIterator() throws IOException { - return Iterators.transform(baseFirehoseFactory.getSplits(), split -> newTaskSpec((InputSplit) split)); + return Iterators.transform(baseFirehoseFactory.getSplits(), this::newTaskSpec); } @VisibleForTesting - ParallelIndexSinglePhaseSubTaskSpec newTaskSpec(InputSplit split) + ParallelIndexSinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { return new ParallelIndexSinglePhaseSubTaskSpec( getId() + "_" + nextSpecId++, @@ -395,7 +402,7 @@ ParallelIndexSinglePhaseSubTaskSpec newTaskSpec(InputSplit split) private static List getSplitsIfSplittable(FirehoseFactory firehoseFactory) throws IOException { if (firehoseFactory instanceof FiniteFirehoseFactory) { - final FiniteFirehoseFactory finiteFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; + final FiniteFirehoseFactory finiteFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; return Lists.newArrayList(finiteFirehoseFactory.getSplits()); } else { throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java index b60c2759de87..810aad1574c4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java @@ -34,11 +34,11 @@ public class ParallelIndexSinglePhaseTuningConfig extends IndexTuningConfig { private static final int DEFAULT_MAX_NUM_BATCH_TASKS = Integer.MAX_VALUE; // unlimited private static final int DEFAULT_MAX_RETRY = 3; - private static final long DEFAULT_TASK_STATUS_CHECKING_PERIOD_MS = 1000; + private static final long DEFAULT_TASK_STATUS_CHECK_PERIOD_MS = 1000; private final int maxNumBatchTasks; private final int maxRetry; - private final long taskStatusCheckingPeriodMs; + private final long taskStatusCheckPeriodMs; public static ParallelIndexSinglePhaseTuningConfig defaultConfig() { @@ -55,9 +55,6 @@ public static ParallelIndexSinglePhaseTuningConfig defaultConfig() null, null, null, - null, - null, - null, null ); } @@ -67,44 +64,40 @@ public ParallelIndexSinglePhaseTuningConfig( @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, - @JsonProperty("rowFlushBoundary") @Nullable Integer rowFlushBoundary_forBackCompatibility, // DEPRECATED @JsonProperty("numShards") @Nullable Integer numShards, @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, - // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. - @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs, @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, - @JsonProperty("publishTimeout") @Nullable Long publishTimeout, // deprecated @JsonProperty("pushTimeout") @Nullable Long pushTimeout, @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("maxNumBatchTasks") @Nullable Integer maxNumBatchTasks, @JsonProperty("maxRetry") @Nullable Integer maxRetry, - @JsonProperty("taskStatusCheckingPeriodMs") @Nullable Integer taskStatusCheckingPeriodMs + @JsonProperty("taskStatusCheckPeriodMs") @Nullable Integer taskStatusCheckPeriodMs ) { super( targetPartitionSize, maxRowsInMemory, maxTotalRows, - rowFlushBoundary_forBackCompatibility, + null, numShards, indexSpec, maxPendingPersists, - buildV9Directly, + null, forceExtendableShardSpecs, false, // ParallelIndexSinglePhaseSupervisorTask can't be used for guaranteed rollup reportParseExceptions, - publishTimeout, + null, pushTimeout, segmentWriteOutMediumFactory ); this.maxNumBatchTasks = maxNumBatchTasks == null ? DEFAULT_MAX_NUM_BATCH_TASKS : maxNumBatchTasks; this.maxRetry = maxRetry == null ? DEFAULT_MAX_RETRY : maxRetry; - this.taskStatusCheckingPeriodMs = taskStatusCheckingPeriodMs == null ? - DEFAULT_TASK_STATUS_CHECKING_PERIOD_MS : - taskStatusCheckingPeriodMs; + this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs == null ? + DEFAULT_TASK_STATUS_CHECK_PERIOD_MS : + taskStatusCheckPeriodMs; } @JsonProperty @@ -120,9 +113,9 @@ public int getMaxRetry() } @JsonProperty - public long getTaskStatusCheckingPeriodMs() + public long getTaskStatusCheckPeriodMs() { - return taskStatusCheckingPeriodMs; + return taskStatusCheckPeriodMs; } @Override @@ -139,7 +132,6 @@ public boolean equals(Object o) Objects.equals(getMaxTotalRows(), that.getMaxTotalRows()) && getMaxPendingPersists() == that.getMaxPendingPersists() && isForceExtendableShardSpecs() == that.isForceExtendableShardSpecs() && - isForceGuaranteedRollup() == that.isForceGuaranteedRollup() && isReportParseExceptions() == that.isReportParseExceptions() && getPushTimeout() == that.getPushTimeout() && Objects.equals(getTargetPartitionSize(), that.getTargetPartitionSize()) && @@ -149,7 +141,7 @@ public boolean equals(Object o) Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()) && maxNumBatchTasks == that.maxNumBatchTasks && maxRetry == that.maxRetry && - taskStatusCheckingPeriodMs == that.taskStatusCheckingPeriodMs; + taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs; } @Override @@ -164,13 +156,12 @@ public int hashCode() getBasePersistDirectory(), getMaxPendingPersists(), isForceExtendableShardSpecs(), - isForceGuaranteedRollup(), isReportParseExceptions(), getPushTimeout(), getSegmentWriteOutMediumFactory(), maxNumBatchTasks, maxRetry, - taskStatusCheckingPeriodMs + taskStatusCheckPeriodMs ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index a85cb46bcf7b..f096b6cc3a0b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -90,31 +90,36 @@ public void start(long taskStatusCheckingPeriod) case SUCCESS: numRunningTasks.decrementAndGet(); iterator.remove(); + monitorEntry.setLastStatus(taskStatus); break; case FAILED: numRunningTasks.decrementAndGet(); + iterator.remove(); + log.warn("task[%s] failed!", taskId); if (monitorEntry.numTries() < maxRetry) { log.info( - "We still have chnaces[%d/%d] to complete. Retrying task[%s]", + "We still have chnaces[%d/%d] to complete. Retrying spec[%s]", monitorEntry.numTries(), maxRetry, - taskId + monitorEntry.spec.getId() ); retry(monitorEntry, taskStatus); } else { log.error( - "task[%s] failed after [%d] tries", - taskId, + "spec[%s] failed after [%d] tries", + monitorEntry.spec.getId(), monitorEntry.numTries() ); - iterator.remove(); monitorEntry.setLastStatus(taskStatus); } break; - default: + case RUNNING: // do nothing + break; + default: + throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getStatusCode(), taskId); } } } @@ -160,7 +165,10 @@ private void retry(MonitorEntry monitorEntry, TaskStatus lastFailedTaskStatus) indexingServiceClient.runTask(task); numRunningTasks.incrementAndGet(); - runningTasks.put(task.getId(), monitorEntry.withNewRunningTask(task, lastFailedTaskStatus)); + runningTasks.put( + task.getId(), + monitorEntry.withNewRunningTask(task, lastFailedTaskStatus) + ); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java index 73eab386ce65..c5b16ea04143 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java @@ -152,6 +152,7 @@ public void testIsReady() throws Exception while (subTaskSpecIterator.hasNext()) { final ParallelIndexSinglePhaseSubTaskSpec spec = subTaskSpecIterator.next(); final ParallelIndexSinglePhaseSubTask subTask = new ParallelIndexSinglePhaseSubTask( + null, spec.getGroupId(), null, spec.getSupervisorTaskId(), @@ -254,7 +255,21 @@ private ParallelIndexSinglePhaseSupervisorTask newTask( getObjectMapper() ), ioConfig, - null + new ParallelIndexSinglePhaseTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + 2, + null, + null + ) ); // set up test tools From cc8bd38027f7d92c084c6895f375e94d69694592 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 21 Mar 2018 18:34:14 -0700 Subject: [PATCH 07/47] add it test --- .../indexer/AbstractITBatchIndexTest.java | 90 +++++++++++++++++++ .../io/druid/tests/indexer/ITIndexerTest.java | 78 ++-------------- .../ITParallelIndexSinglePhaseTest.java | 46 ++++++++++ .../druid/tests/indexer/ITUnionQueryTest.java | 2 +- .../batch_index/wikipedia_index_data1.json | 3 + .../batch_index/wikipedia_index_data2.json | 3 + .../batch_index/wikipedia_index_data3.json | 4 + .../indexer/wikipedia_index_data.json | 10 --- .../indexer/wikipedia_index_task.json | 4 +- ...a_parallel_single_phase_index_queries.json | 71 +++++++++++++++ ...edia_parallel_single_phase_index_task.json | 66 ++++++++++++++ 11 files changed, 295 insertions(+), 82 deletions(-) create mode 100644 integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java create mode 100644 integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexSinglePhaseTest.java create mode 100644 integration-tests/src/test/resources/data/batch_index/wikipedia_index_data1.json create mode 100644 integration-tests/src/test/resources/data/batch_index/wikipedia_index_data2.json create mode 100644 integration-tests/src/test/resources/data/batch_index/wikipedia_index_data3.json delete mode 100644 integration-tests/src/test/resources/indexer/wikipedia_index_data.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_queries.json create mode 100644 integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java new file mode 100644 index 000000000000..661018f5529b --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -0,0 +1,90 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.tests.indexer; + +import com.google.inject.Inject; +import io.druid.java.util.common.logger.Logger; +import io.druid.testing.IntegrationTestingConfig; +import io.druid.testing.clients.ClientInfoResourceTestClient; +import io.druid.testing.utils.RetryUtil; +import org.junit.Assert; + +import java.io.IOException; +import java.util.List; + +public class AbstractITBatchIndexTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class); + + @Inject + IntegrationTestingConfig config; + + @Inject + ClientInfoResourceTestClient clientInfoResourceTestClient; + + void doIndexTestTest( + String dataSource, + String indexTaskFilePath, + String queryFilePath + ) throws IOException + { + submitTaskAndWait(indexTaskFilePath, dataSource); + try { + queryHelper.testQueriesFromFile(queryFilePath, 2); + + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + void doReindexTest( + String reindexDataSource, + String reindexTaskFilePath, + String queryFilePath + ) throws IOException + { + submitTaskAndWait(reindexTaskFilePath, reindexDataSource); + try { + queryHelper.testQueriesFromFile(queryFilePath, 2); + // verify excluded dimension is not reIndexed + final List dimensions = clientInfoResourceTestClient.getDimensions( + reindexDataSource, + "2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z" + ); + Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot")); + } + catch (Exception e) { + LOG.error(e, "Error while testing"); + throw new RuntimeException(e); + } + } + + private void submitTaskAndWait(String indexTaskFilePath, String dataSourceName) throws IOException + { + final String taskID = indexer.submitTask(getTaskAsString(indexTaskFilePath)); + LOG.info("TaskID for loading index task %s", taskID); + indexer.waitUntilTaskCompletes(taskID); + + RetryUtil.retryUntilTrue( + () -> coordinator.areSegmentsLoaded(dataSourceName), "Segment Load" + ); + } +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java index 662d6d872e20..89d1b79ccc24 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITIndexerTest.java @@ -19,97 +19,37 @@ package io.druid.tests.indexer; -import com.google.common.base.Throwables; -import com.google.inject.Inject; -import io.druid.java.util.common.logger.Logger; -import io.druid.testing.IntegrationTestingConfig; -import io.druid.testing.clients.ClientInfoResourceTestClient; import io.druid.testing.guice.DruidTestModuleFactory; -import io.druid.testing.utils.RetryUtil; -import org.junit.Assert; import org.testng.annotations.Guice; import org.testng.annotations.Test; -import java.util.List; -import java.util.concurrent.Callable; - @Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITIndexerTest extends AbstractIndexerTest +public class ITIndexerTest extends AbstractITBatchIndexTest { - private static final Logger LOG = new Logger(ITIndexerTest.class); private static String INDEX_TASK = "/indexer/wikipedia_index_task.json"; private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; private static String INDEX_DATASOURCE = "wikipedia_index_test"; private static String REINDEX_TASK = "/indexer/wikipedia_reindex_task.json"; private static String REINDEX_DATASOURCE = "wikipedia_reindex_test"; - @Inject - private IntegrationTestingConfig config; - - @Inject - ClientInfoResourceTestClient clientInfoResourceTestClient; - @Test public void testIndexData() throws Exception { - loadData(); try { - queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2); - reIndexData(); - queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2); - // verify excluded dimension is not reIndexed - final List dimensions = clientInfoResourceTestClient.getDimensions( + doIndexTestTest( + INDEX_DATASOURCE, + INDEX_TASK, + INDEX_QUERIES_RESOURCE + ); + doReindexTest( REINDEX_DATASOURCE, - "2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z" + REINDEX_TASK, + INDEX_QUERIES_RESOURCE ); - Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot")); - } - catch (Exception e) { - LOG.error(e, "Error while testing"); - throw Throwables.propagate(e); } finally { unloadAndKillData(INDEX_DATASOURCE); unloadAndKillData(REINDEX_DATASOURCE); - } - - } - - private void loadData() throws Exception - { - final String taskID = indexer.submitTask(getTaskAsString(INDEX_TASK)); - LOG.info("TaskID for loading index task %s", taskID); - indexer.waitUntilTaskCompletes(taskID); - - RetryUtil.retryUntilTrue( - new Callable() - { - @Override - public Boolean call() - { - return coordinator.areSegmentsLoaded(INDEX_DATASOURCE); - } - }, "Segment Load" - ); - } - - private void reIndexData() throws Exception - { - final String taskID = indexer.submitTask(getTaskAsString(REINDEX_TASK)); - LOG.info("TaskID for loading index task %s", taskID); - indexer.waitUntilTaskCompletes(taskID); - - RetryUtil.retryUntilTrue( - new Callable() - { - @Override - public Boolean call() - { - return coordinator.areSegmentsLoaded(REINDEX_DATASOURCE); - } - }, "Segment Load" - ); } - } diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexSinglePhaseTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexSinglePhaseTest.java new file mode 100644 index 000000000000..0aea7a73e4c1 --- /dev/null +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexSinglePhaseTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.tests.indexer; + +import io.druid.testing.guice.DruidTestModuleFactory; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class ITParallelIndexSinglePhaseTest extends AbstractITBatchIndexTest +{ + private static String INDEX_TASK = "/indexer/wikipedia_parallel_single_phase_index_task.json"; + private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_single_phase_index_queries.json"; + private static String INDEX_DATASOURCE = "wikipedia_parallel_single_phase_index_test"; + + @Test + public void testIndexData() throws Exception + { + try { + doIndexTestTest( + INDEX_DATASOURCE, + INDEX_TASK, + INDEX_QUERIES_RESOURCE + ); + } + finally { + unloadAndKillData(INDEX_DATASOURCE); + } + } +} diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java index 4a792c2909b2..f0209b71778e 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java @@ -47,7 +47,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest private static final Logger LOG = new Logger(ITUnionQueryTest.class); private static final String UNION_TASK_RESOURCE = "/indexer/wikipedia_union_index_task.json"; private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName"; - private static final String UNION_DATA_FILE = "/indexer/wikipedia_index_data.json"; + private static final String UNION_DATA_FILE = "/data/batch_index/wikipedia_index_data1.json"; private static final String UNION_QUERIES_RESOURCE = "/indexer/union_queries.json"; private static final String UNION_DATASOURCE = "wikipedia_index_test"; diff --git a/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data1.json b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data1.json new file mode 100644 index 000000000000..169796cd7468 --- /dev/null +++ b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data1.json @@ -0,0 +1,3 @@ +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} \ No newline at end of file diff --git a/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data2.json b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data2.json new file mode 100644 index 000000000000..62e270113d51 --- /dev/null +++ b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data2.json @@ -0,0 +1,3 @@ +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} \ No newline at end of file diff --git a/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data3.json b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data3.json new file mode 100644 index 000000000000..28e0762f84cf --- /dev/null +++ b/integration-tests/src/test/resources/data/batch_index/wikipedia_index_data3.json @@ -0,0 +1,4 @@ +{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_data.json b/integration-tests/src/test/resources/indexer/wikipedia_index_data.json deleted file mode 100644 index b186657dbcf0..000000000000 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_data.json +++ /dev/null @@ -1,10 +0,0 @@ -{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} -{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} -{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} -{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} -{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} -{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} -{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} -{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} -{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} -{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json index 15e0de89560c..af7b98a6a716 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_index_task.json @@ -58,8 +58,8 @@ "type": "index", "firehose": { "type": "local", - "baseDir": "/resources/indexer", - "filter": "wikipedia_index_data.json" + "baseDir": "/resources/data/batch_index", + "filter": "wikipedia_index_data*" } }, "tuningConfig": { diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_queries.json new file mode 100644 index 000000000000..3e3138020aed --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_queries.json @@ -0,0 +1,71 @@ +[ + { + "description": "timeseries, 1 agg, all", + "query":{ + "queryType" : "timeBoundary", + "dataSource": "wikipedia_parallel_single_phase_index_test" + }, + "expectedResults":[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-09-01T12:41:27.000Z" + } + } + ] + }, + + { + "description":"having spec on post aggregation", + "query":{ + "queryType":"groupBy", + "dataSource":"wikipedia_parallel_single_phase_index_test", + "granularity":"day", + "dimensions":[ + "page" + ], + "filter":{ + "type":"selector", + "dimension":"language", + "value":"zh" + }, + "aggregations":[ + { + "type":"count", + "name":"rows" + }, + { + "type":"longSum", + "fieldName":"added", + "name":"added_count" + } + ], + "postAggregations": [ + { + "type":"arithmetic", + "name":"added_count_times_ten", + "fn":"*", + "fields":[ + {"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"}, + {"type":"constant", "name":"const", "value":10} + ] + } + ], + "having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000}, + "intervals":[ + "2013-08-31T00:00/2013-09-01T00:00" + ] + }, + "expectedResults":[ { + "version" : "v1", + "timestamp" : "2013-08-31T00:00:00.000Z", + "event" : { + "added_count_times_ten" : 9050.0, + "page" : "Crimson Typhoon", + "added_count" : 905, + "rows" : 1 + } + } ] + } +] \ No newline at end of file diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json new file mode 100644 index 000000000000..7495a61ca2e5 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json @@ -0,0 +1,66 @@ +{ + "type": "parallelIndexSinglePhase", + "spec": { + "dataSchema": { + "dataSource": "wikipedia_parallel_single_phase_index_test", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + {"type": "string", "name": "language", "createBitmapIndex": false}, + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "parallelIndexSinglePhase", + "firehose": { + "type": "local", + "baseDir": "/resources/data/batch_index", + "filter": "wikipedia_index_data*" + } + } + } +} \ No newline at end of file From 2d2347b622f41830cc2a21e937f49a6bd395f97a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 22 Mar 2018 17:53:08 -0700 Subject: [PATCH 08/47] use chat handler --- .../java/io/druid/indexer/TaskStatusPlus.java | 24 +- .../druid/indexing/kafka/KafkaIndexTask.java | 18 +- .../indexing/kafka/KafkaIndexTaskClient.java | 408 +++--------------- .../kafka/KafkaIndexTaskClientFactory.java | 6 +- .../indexing/kafka/KafkaIndexTaskModule.java | 6 + .../kafka/KafkaIndexTaskClientTest.java | 5 +- .../indexing/common/IndexTaskClient.java | 375 ++++++++++++++++ .../io/druid/indexing/common/TaskStatus.java | 33 +- .../task/ClientBasedTaskInfoProvider.java | 55 +++ .../common/task/IndexTaskClientFactory.java | 29 ++ .../common/task/PushedSegmentsReport.java | 27 +- ... => SinglePhaseParallelIndexIOConfig.java} | 6 +- ...inglePhaseParallelIndexIngestionSpec.java} | 20 +- ...a => SinglePhaseParallelIndexSubTask.java} | 55 ++- ... SinglePhaseParallelIndexSubTaskSpec.java} | 20 +- ...nglePhaseParallelIndexSupervisorTask.java} | 166 +++++-- .../SinglePhaseParallelIndexTaskClient.java | 80 ++++ ...lePhaseParallelIndexTaskClientFactory.java | 66 +++ ...SinglePhaseParallelIndexTuningConfig.java} | 51 ++- .../io/druid/indexing/common/task/Task.java | 4 +- .../indexing/common/task/TaskMonitor.java | 34 +- .../overlord/http/OverlordResource.java | 63 ++- .../overlord/http/TaskStatusResponse.java | 8 +- .../io/druid/indexing/common/TestUtils.java | 8 +- ...eParallelIndexSupervisorTaskSerdeTest.java | 156 +++++++ ...PhaseParallelIndexSupervisorTaskTest.java} | 207 +++++++-- .../indexing/common/task/TaskMonitorTest.java | 45 +- .../overlord/http/OverlordResourceTest.java | 37 +- .../indexing/overlord/http/OverlordTest.java | 6 +- .../clients/OverlordResourceTestClient.java | 12 +- ...va => ITSinglePhaseParallelIndexTest.java} | 2 +- ...edia_parallel_single_phase_index_task.json | 4 +- .../indexing/HttpIndexingServiceClient.java | 310 +++++++++++++ .../indexing/IndexingServiceClient.java | 262 +---------- .../io/druid/client/indexing/TaskStatus.java | 18 +- .../client/indexing/TaskStatusResponse.java | 7 +- .../io/druid/guice/http/HttpClientModule.java | 6 +- .../realtime/firehose/ChatHandlers.java | 59 +++ .../indexing/NoopIndexingServiceClient.java | 117 +++++ .../DruidCoordinatorSegmentMergerTest.java | 5 +- .../DruidCoordinatorSegmentCompactorTest.java | 3 +- .../java/io/druid/cli/CliCoordinator.java | 3 +- .../java/io/druid/cli/CliMiddleManager.java | 7 + .../main/java/io/druid/cli/CliOverlord.java | 10 +- .../src/main/java/io/druid/cli/CliPeon.java | 11 +- 45 files changed, 1959 insertions(+), 895 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java rename api/src/main/java/io/druid/indexer/TaskReport.java => indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java (66%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ParallelIndexSinglePhaseIOConfig.java => SinglePhaseParallelIndexIOConfig.java} (90%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ParallelIndexSinglePhaseIngestionSpec.java => SinglePhaseParallelIndexIngestionSpec.java} (69%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ParallelIndexSinglePhaseSubTask.java => SinglePhaseParallelIndexSubTask.java} (91%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ParallelIndexSinglePhaseSubTaskSpec.java => SinglePhaseParallelIndexSubTaskSpec.java} (72%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ParallelIndexSinglePhaseSupervisorTask.java => SinglePhaseParallelIndexSupervisorTask.java} (69%) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClientFactory.java rename indexing-service/src/main/java/io/druid/indexing/common/task/{ParallelIndexSinglePhaseTuningConfig.java => SinglePhaseParallelIndexTuningConfig.java} (78%) create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskSerdeTest.java rename indexing-service/src/test/java/io/druid/indexing/common/task/{ParallelIndexSinglePhaseSupervisorTaskTest.java => SinglePhaseParallelIndexSupervisorTaskTest.java} (64%) rename integration-tests/src/test/java/io/druid/tests/indexer/{ITParallelIndexSinglePhaseTest.java => ITSinglePhaseParallelIndexTest.java} (96%) create mode 100644 server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java create mode 100644 server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java create mode 100644 server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java diff --git a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java index d8d93d6055d5..6f3564a6e871 100644 --- a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java +++ b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java @@ -108,6 +108,12 @@ public TaskLocation getLocation() return location; } + @JsonProperty + public String getDataSource() + { + return dataSource; + } + @Override public boolean equals(Object o) { @@ -146,11 +152,19 @@ public int hashCode() { return Objects.hash(id, type, createdTime, queueInsertionTime, state, duration, location); } - - @JsonProperty - public String getDataSource() + + @Override + public String toString() { - return dataSource; + return "TaskStatusPlus{" + + "id=" + id + + ", type=" + type + + ", createdTime=" + createdTime + + ", queueInsertionTime=" + queueInsertionTime + + ", state=" + state + + ", duration=" + duration + + ", location=" + location + + ", dataSource=" + dataSource + + "}"; } - } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index a325948a6506..45570cdfd01a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -90,14 +90,10 @@ import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import io.druid.segment.realtime.firehose.ChatHandler; import io.druid.segment.realtime.firehose.ChatHandlerProvider; +import io.druid.segment.realtime.firehose.ChatHandlers; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.ForbiddenException; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceAction; -import io.druid.server.security.ResourceType; import io.druid.timeline.DataSegment; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -1340,17 +1336,7 @@ public boolean canRestore() */ private Access authorizationCheck(final HttpServletRequest req, Action action) { - ResourceAction resourceAction = new ResourceAction( - new Resource(dataSchema.getDataSource(), ResourceType.DATASOURCE), - action - ); - - Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); - if (!access.isAllowed()) { - throw new ForbiddenException(access.toString()); - } - - return access; + return ChatHandlers.authorizationCheck(req, action, dataSchema.getDataSource(), authorizerMapper); } @VisibleForTesting diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index c869ed4665bd..419bfd26bc9b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -21,79 +21,30 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; -import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import io.druid.indexer.TaskLocation; -import io.druid.indexing.common.RetryPolicy; -import io.druid.indexing.common.RetryPolicyConfig; -import io.druid.indexing.common.RetryPolicyFactory; +import io.druid.indexing.common.IndexTaskClient; import io.druid.indexing.common.TaskInfoProvider; -import io.druid.indexing.common.TaskStatus; -import io.druid.java.util.common.IAE; -import io.druid.java.util.common.IOE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.emitter.EmittingLogger; import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.Request; -import io.druid.java.util.http.client.response.FullResponseHandler; import io.druid.java.util.http.client.response.FullResponseHolder; -import io.druid.segment.realtime.firehose.ChatHandlerResource; -import org.jboss.netty.channel.ChannelException; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; import org.joda.time.Duration; -import org.joda.time.Period; -import javax.ws.rs.core.MediaType; +import javax.annotation.Nullable; import java.io.IOException; -import java.net.Socket; -import java.net.URI; import java.util.Map; import java.util.TreeMap; -import java.util.concurrent.Callable; -public class KafkaIndexTaskClient +public class KafkaIndexTaskClient extends IndexTaskClient { - public static class NoTaskLocationException extends RuntimeException - { - public NoTaskLocationException(String message) - { - super(message); - } - } - - public static class TaskNotRunnableException extends RuntimeException - { - public TaskNotRunnableException(String message) - { - super(message); - } - } - - public static final int MAX_RETRY_WAIT_SECONDS = 10; - - private static final int MIN_RETRY_WAIT_SECONDS = 2; private static final EmittingLogger log = new EmittingLogger(KafkaIndexTaskClient.class); - private static final String BASE_PATH = "/druid/worker/v1/chat"; - private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5; - private static final TreeMap EMPTY_TREE_MAP = new TreeMap(); - - private final HttpClient httpClient; - private final ObjectMapper jsonMapper; - private final TaskInfoProvider taskInfoProvider; - private final Duration httpTimeout; - private final RetryPolicyFactory retryPolicyFactory; - private final ListeningExecutorService executorService; - private final long numRetries; + private static final TreeMap> EMPTY_TREE_MAP = new TreeMap<>(); public KafkaIndexTaskClient( HttpClient httpClient, @@ -105,27 +56,7 @@ public KafkaIndexTaskClient( long numRetries ) { - this.httpClient = httpClient; - this.jsonMapper = jsonMapper; - this.taskInfoProvider = taskInfoProvider; - this.httpTimeout = httpTimeout; - this.numRetries = numRetries; - this.retryPolicyFactory = createRetryPolicyFactory(); - - this.executorService = MoreExecutors.listeningDecorator( - Execs.multiThreaded( - numThreads, - StringUtils.format( - "KafkaIndexTaskClient-%s-%%d", - dataSource - ) - ) - ); - } - - public void close() - { - executorService.shutdownNow(); + super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries); } public boolean stop(final String id, final boolean publish) @@ -133,10 +64,10 @@ public boolean stop(final String id, final boolean publish) log.debug("Stop task[%s] publish[%s]", id, publish); try { - final FullResponseHolder response = submitRequest( + final FullResponseHolder response = submitRequestWithEmptyContent( id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true ); - return response.getStatus().getCode() / 100 == 2; + return isSuccess(response); } catch (NoTaskLocationException e) { return false; @@ -156,10 +87,11 @@ public boolean resume(final String id) log.debug("Resume task[%s]", id); try { - final FullResponseHolder response = submitRequest(id, HttpMethod.POST, "resume", null, true); - return response.getStatus().getCode() / 100 == 2; + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true); + return isSuccess(response); } - catch (NoTaskLocationException e) { + catch (NoTaskLocationException | IOException e) { + log.warn(e, "Exception while stopping task [%s]", id); return false; } } @@ -174,7 +106,7 @@ public Map pause(final String id, final long timeout) log.debug("Pause task[%s] timeout[%d]", id, timeout); try { - final FullResponseHolder response = submitRequest( + final FullResponseHolder response = submitRequestWithEmptyContent( id, HttpMethod.POST, "pause", @@ -184,18 +116,17 @@ public Map pause(final String id, final long timeout) if (response.getStatus().equals(HttpResponseStatus.OK)) { log.info("Task [%s] paused successfully", id); - return jsonMapper.readValue(response.getContent(), new TypeReference>() + return deserialize(response.getContent(), new TypeReference>() { }); } - final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); while (true) { if (getStatus(id) == KafkaIndexTask.Status.PAUSED) { return getCurrentOffsets(id, true); } - final Duration delay = retryPolicy.getAndIncrementRetryDelay(); + final Duration delay = newRetryPolicy().getAndIncrementRetryDelay(); if (delay == null) { log.error("Task [%s] failed to pause, aborting", id); throw new ISE("Task [%s] failed to pause, aborting", id); @@ -225,8 +156,8 @@ public KafkaIndexTask.Status getStatus(final String id) log.debug("GetStatus task[%s]", id); try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "status", null, true); - return jsonMapper.readValue(response.getContent(), KafkaIndexTask.Status.class); + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true); + return deserialize(response.getContent(), KafkaIndexTask.Status.class); } catch (NoTaskLocationException e) { return KafkaIndexTask.Status.NOT_STARTED; @@ -236,15 +167,16 @@ public KafkaIndexTask.Status getStatus(final String id) } } + @Nullable public DateTime getStartTime(final String id) { log.debug("GetStartTime task[%s]", id); try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "time/start", null, true); + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true); return response.getContent() == null || response.getContent().isEmpty() ? null - : jsonMapper.readValue(response.getContent(), DateTime.class); + : deserialize(response.getContent(), DateTime.class); } catch (NoTaskLocationException e) { return null; @@ -259,8 +191,14 @@ public Map getCurrentOffsets(final String id, final boolean retry log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry); try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/current", null, retry); - return jsonMapper.readValue(response.getContent(), new TypeReference>() + final FullResponseHolder response = submitRequestWithEmptyContent( + id, + HttpMethod.GET, + "offsets/current", + null, + retry + ); + return deserialize(response.getContent(), new TypeReference>() { }); } @@ -276,10 +214,13 @@ public TreeMap> getCheckpoints(final String id, fina { log.debug("GetCheckpoints task[%s] retry[%s]", id, retry); try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "checkpoints", null, retry); - return jsonMapper.readValue(response.getContent(), new TypeReference>>() - { - }); + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry); + return deserialize( + response.getContent(), + new TypeReference>>() + { + } + ); } catch (NoTaskLocationException e) { return EMPTY_TREE_MAP; @@ -294,9 +235,7 @@ public ListenableFuture>> getCheckpointsAsyn final boolean retry ) { - return executorService.submit( - () -> getCheckpoints(id, retry) - ); + return doAsync(() -> getCheckpoints(id, retry)); } public Map getEndOffsets(final String id) @@ -304,8 +243,8 @@ public Map getEndOffsets(final String id) log.debug("GetEndOffsets task[%s]", id); try { - final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/end", null, true); - return jsonMapper.readValue(response.getContent(), new TypeReference>() + final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true); + return deserialize(response.getContent(), new TypeReference>() { }); } @@ -327,15 +266,15 @@ public boolean setEndOffsets( log.debug("SetEndOffsets task[%s] endOffsets[%s] resume[%s] finalize[%s]", id, endOffsets, resume, finalize); try { - final FullResponseHolder response = submitRequest( + final FullResponseHolder response = submitJsonRequest( id, HttpMethod.POST, "offsets/end", StringUtils.format("resume=%s&finish=%s", resume, finalize), - jsonMapper.writeValueAsBytes(endOffsets), + serialize(endOffsets), true ); - return response.getStatus().getCode() / 100 == 2; + return isSuccess(response); } catch (NoTaskLocationException e) { return false; @@ -347,30 +286,12 @@ public boolean setEndOffsets( public ListenableFuture stopAsync(final String id, final boolean publish) { - return executorService.submit( - new Callable() - { - @Override - public Boolean call() - { - return stop(id, publish); - } - } - ); + return doAsync(() -> stop(id, publish)); } public ListenableFuture resumeAsync(final String id) { - return executorService.submit( - new Callable() - { - @Override - public Boolean call() - { - return resume(id); - } - } - ); + return doAsync(() -> resume(id)); } public ListenableFuture> pauseAsync(final String id) @@ -380,261 +301,36 @@ public ListenableFuture> pauseAsync(final String id) public ListenableFuture> pauseAsync(final String id, final long timeout) { - return executorService.submit( - new Callable>() - { - @Override - public Map call() - { - return pause(id, timeout); - } - } - ); + return doAsync(() -> pause(id, timeout)); } public ListenableFuture getStatusAsync(final String id) { - return executorService.submit( - new Callable() - { - @Override - public KafkaIndexTask.Status call() - { - return getStatus(id); - } - } - ); + return doAsync(() -> getStatus(id)); } public ListenableFuture getStartTimeAsync(final String id) { - return executorService.submit( - new Callable() - { - @Override - public DateTime call() - { - return getStartTime(id); - } - } - ); + return doAsync(() -> getStartTime(id)); } public ListenableFuture> getCurrentOffsetsAsync(final String id, final boolean retry) { - return executorService.submit( - new Callable>() - { - @Override - public Map call() - { - return getCurrentOffsets(id, retry); - } - } - ); + return doAsync(() -> getCurrentOffsets(id, retry)); } public ListenableFuture> getEndOffsetsAsync(final String id) { - return executorService.submit( - new Callable>() - { - @Override - public Map call() - { - return getEndOffsets(id); - } - } - ); + return doAsync(() -> getEndOffsets(id)); } public ListenableFuture setEndOffsetsAsync( - final String id, final Map endOffsets, final boolean resume, final boolean finalize - ) - { - return executorService.submit( - new Callable() - { - @Override - public Boolean call() - { - return setEndOffsets(id, endOffsets, resume, finalize); - } - } - ); - } - - @VisibleForTesting - RetryPolicyFactory createRetryPolicyFactory() - { - // Retries [numRetries] times before giving up; this should be set long enough to handle any temporary - // unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in - // the middle of persisting to disk and doesn't respond immediately. - return new RetryPolicyFactory( - new RetryPolicyConfig() - .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS)) - .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS)) - .setMaxRetryCount(numRetries) - ); - } - - @VisibleForTesting - void checkConnection(String host, int port) throws IOException - { - new Socket(host, port).close(); - } - - private FullResponseHolder submitRequest(String id, HttpMethod method, String pathSuffix, String query, boolean retry) - { - return submitRequest(id, method, pathSuffix, query, new byte[0], retry); - } - - private FullResponseHolder submitRequest( - String id, - HttpMethod method, - String pathSuffix, - String query, - byte[] content, - boolean retry + final String id, + final Map endOffsets, + final boolean resume, + final boolean finalize ) { - final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); - while (true) { - FullResponseHolder response = null; - Request request = null; - TaskLocation location = TaskLocation.unknown(); - String path = StringUtils.format("%s/%s/%s", BASE_PATH, id, pathSuffix); - - Optional status = taskInfoProvider.getTaskStatus(id); - if (!status.isPresent() || !status.get().isRunnable()) { - throw new TaskNotRunnableException(StringUtils.format( - "Aborting request because task [%s] is not runnable", - id - )); - } - - String host = location.getHost(); - String scheme = ""; - int port = -1; - - try { - location = taskInfoProvider.getTaskLocation(id); - if (location.equals(TaskLocation.unknown())) { - throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", id)); - } - - host = location.getHost(); - scheme = location.getTlsPort() >= 0 ? "https" : "http"; - port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort(); - - // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently - // for tasks that happen to still be starting up, so test the connection first to keep the logs clean. - checkConnection(host, port); - - try { - URI serviceUri = new URI( - scheme, - null, - host, - port, - path, - query, - null - ); - request = new Request(method, serviceUri.toURL()); - - // used to validate that we are talking to the correct worker - request.addHeader(ChatHandlerResource.TASK_ID_HEADER, id); - - if (content.length > 0) { - request.setContent(MediaType.APPLICATION_JSON, content); - } - - log.debug("HTTP %s: %s", method.getName(), serviceUri.toString()); - response = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8), httpTimeout).get(); - } - catch (Exception e) { - Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); - Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); - throw Throwables.propagate(e); - } - - int responseCode = response.getStatus().getCode(); - if (responseCode / 100 == 2) { - return response; - } else if (responseCode == 400) { // don't bother retrying if it's a bad request - throw new IAE("Received 400 Bad Request with body: %s", response.getContent()); - } else { - throw new IOE("Received status [%d]", responseCode); - } - } - catch (IOException | ChannelException e) { - - // Since workers are free to move tasks around to different ports, there is a chance that a task may have been - // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header - // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the - // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then - // we will wait for a short period then retry the request indefinitely, expecting the task's location to - // eventually be updated. - - final Duration delay; - if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) { - String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER); - if (headerId != null && !headerId.equals(id)) { - log.warn( - "Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s", - id, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS - ); - delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS); - } else { - delay = retryPolicy.getAndIncrementRetryDelay(); - } - } else { - delay = retryPolicy.getAndIncrementRetryDelay(); - } - String urlForLog = (request != null - ? request.getUrl().toString() - : StringUtils.format( - "%s://%s:%d%s", - scheme, - host, - port, - path - )); - if (!retry) { - // if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was - // for informational purposes only) so don't log a scary stack trace - log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage()); - Throwables.propagate(e); - } else if (delay == null) { - log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog); - Throwables.propagate(e); - } else { - try { - final long sleepTime = delay.getMillis(); - log.debug( - "Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])", - (response != null ? response.getStatus().getCode() : "no response"), - urlForLog, - new Duration(sleepTime).toString(), - (response != null ? response.getContent() : e.getMessage()) - ); - Thread.sleep(sleepTime); - } - catch (InterruptedException e2) { - Throwables.propagate(e2); - } - } - } - catch (NoTaskLocationException e) { - log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or " - + "may have already completed", id); - throw e; - } - catch (Exception e) { - log.warn(e, "Exception while sending request"); - throw e; - } - } + return doAsync(() -> setEndOffsets(id, endOffsets, resume, finalize)); } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index da4d8c369f94..1b84106320e5 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -21,13 +21,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; -import io.druid.java.util.http.client.HttpClient; import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Json; import io.druid.indexing.common.TaskInfoProvider; +import io.druid.indexing.common.task.IndexTaskClientFactory; +import io.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -public class KafkaIndexTaskClientFactory +public class KafkaIndexTaskClientFactory implements IndexTaskClientFactory { private HttpClient httpClient; private ObjectMapper mapper; @@ -42,6 +43,7 @@ public KafkaIndexTaskClientFactory( this.mapper = mapper; } + @Override public KafkaIndexTaskClient build( TaskInfoProvider taskInfoProvider, String dataSource, diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java index b0af2fdc8a51..7015842752b1 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -24,6 +24,9 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import com.google.inject.TypeLiteral; +import io.druid.guice.LazySingleton; +import io.druid.indexing.common.task.IndexTaskClientFactory; import io.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; import io.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; import io.druid.initialization.DruidModule; @@ -50,5 +53,8 @@ public List getJacksonModules() @Override public void configure(Binder binder) { + binder.bind(new TypeLiteral>(){}) + .to(KafkaIndexTaskClientFactory.class) + .in(LazySingleton.class); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index 401c98d8f186..523864966a64 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -28,6 +28,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.druid.indexer.TaskLocation; +import io.druid.indexing.common.IndexTaskClient.TaskNotRunnableException; import io.druid.indexing.common.TaskInfoProvider; import io.druid.indexing.common.TaskStatus; import io.druid.jackson.DefaultObjectMapper; @@ -149,7 +150,7 @@ public void testNoTaskLocation() verifyAll(); } - @Test(expected = KafkaIndexTaskClient.TaskNotRunnableException.class) + @Test(expected = TaskNotRunnableException.class) public void testTaskNotRunnableException() { reset(taskInfoProvider); @@ -992,7 +993,7 @@ public TestableKafkaIndexTaskClient( } @Override - void checkConnection(String host, int port) + protected void checkConnection(String host, int port) { } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java new file mode 100644 index 000000000000..6edad44f5040 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java @@ -0,0 +1,375 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.indexer.TaskLocation; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.IOE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.emitter.EmittingLogger; +import io.druid.java.util.http.client.HttpClient; +import io.druid.java.util.http.client.Request; +import io.druid.java.util.http.client.response.FullResponseHandler; +import io.druid.java.util.http.client.response.FullResponseHolder; +import io.druid.segment.realtime.firehose.ChatHandlerResource; +import org.jboss.netty.channel.ChannelException; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import javax.ws.rs.core.MediaType; +import java.io.IOException; +import java.net.Socket; +import java.net.URI; +import java.util.concurrent.Callable; + +/** + * Abstract class to communicate with index tasks via HTTP. This class provides interfaces to serialize/deserialize + * data and send an HTTP request. + */ +public abstract class IndexTaskClient implements AutoCloseable +{ + public static class NoTaskLocationException extends RuntimeException + { + public NoTaskLocationException(String message) + { + super(message); + } + } + + public static class TaskNotRunnableException extends RuntimeException + { + public TaskNotRunnableException(String message) + { + super(message); + } + } + + public static final int MAX_RETRY_WAIT_SECONDS = 10; + + private static final EmittingLogger log = new EmittingLogger(IndexTaskClient.class); + private static final String BASE_PATH = "/druid/worker/v1/chat"; + private static final int MIN_RETRY_WAIT_SECONDS = 2; + private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5; + + private final HttpClient httpClient; + private final ObjectMapper objectMapper; + private final TaskInfoProvider taskInfoProvider; + private final Duration httpTimeout; + private final RetryPolicyFactory retryPolicyFactory; + private final ListeningExecutorService executorService; + + public IndexTaskClient( + HttpClient httpClient, + ObjectMapper objectMapper, + TaskInfoProvider taskInfoProvider, + Duration httpTimeout, + String callerId, + int numThreads, + long numRetries + ) + { + this.httpClient = httpClient; + this.objectMapper = objectMapper; + this.taskInfoProvider = taskInfoProvider; + this.httpTimeout = httpTimeout; + this.retryPolicyFactory = initializeRetryPolicyFactory(numRetries); + this.executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded( + numThreads, + StringUtils.format( + "IndexTaskClient-%s-%%d", + callerId + ) + ) + ); + } + + private static RetryPolicyFactory initializeRetryPolicyFactory(long numRetries) + { + // Retries [numRetries] times before giving up; this should be set long enough to handle any temporary + // unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in + // the middle of persisting to disk and doesn't respond immediately. + return new RetryPolicyFactory( + new RetryPolicyConfig() + .setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS)) + .setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS)) + .setMaxRetryCount(numRetries) + ); + } + + protected HttpClient getHttpClient() + { + return httpClient; + } + + protected RetryPolicy newRetryPolicy() + { + return retryPolicyFactory.makeRetryPolicy(); + } + + protected T deserialize(String content, TypeReference typeReference) throws IOException + { + return objectMapper.readValue(content, typeReference); + } + + protected T deserialize(String content, Class typeReference) throws IOException + { + return objectMapper.readValue(content, typeReference); + } + + protected byte[] serialize(Object value) throws JsonProcessingException + { + return objectMapper.writeValueAsBytes(value); + } + + protected ListenableFuture doAsync(Callable callable) + { + return executorService.submit(callable); + } + + protected boolean isSuccess(FullResponseHolder responseHolder) + { + return responseHolder.getStatus().getCode() / 100 == 2; + } + + @VisibleForTesting + protected void checkConnection(String host, int port) throws IOException + { + new Socket(host, port).close(); + } + + protected FullResponseHolder submitRequestWithEmptyContent( + String taskId, + HttpMethod method, + String pathSuffix, + @Nullable String query, + boolean retry + ) throws IOException, ChannelException, NoTaskLocationException + { + return submitRequest(taskId, null, method, pathSuffix, query, new byte[0], retry); + } + + /** + * To use this method, {@link #objectMapper} should be a jsonMapper. + */ + protected FullResponseHolder submitJsonRequest( + String taskId, + HttpMethod method, + String pathSuffix, + @Nullable String query, + byte[] content, + boolean retry + ) throws IOException, ChannelException, NoTaskLocationException + { + return submitRequest(taskId, MediaType.APPLICATION_JSON, method, pathSuffix, query, content, retry); + } + + /** + * To use this method, {@link #objectMapper} should be a smileMapper. + */ + protected FullResponseHolder submitSmilRequest( + String taskId, + HttpMethod method, + String pathSuffix, + @Nullable String query, + byte[] content, + boolean retry + ) throws IOException, ChannelException, NoTaskLocationException + { + return submitRequest(taskId, SmileMediaTypes.APPLICATION_JACKSON_SMILE, method, pathSuffix, query, content, retry); + } + + /** + * Sends an HTTP request to the task of the specified {@code taskId} and returns a response if it succeeded. + */ + private FullResponseHolder submitRequest( + String taskId, + @Nullable String mediaType, // nullable if content is empty + HttpMethod method, + String pathSuffix, + @Nullable String query, + byte[] content, + boolean retry + ) throws IOException, ChannelException, NoTaskLocationException + { + final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); + + while (true) { + FullResponseHolder response = null; + Request request = null; + TaskLocation location = TaskLocation.unknown(); + String path = StringUtils.format("%s/%s/%s", BASE_PATH, taskId, pathSuffix); + + Optional status = taskInfoProvider.getTaskStatus(taskId); + if (!status.isPresent() || !status.get().isRunnable()) { + throw new TaskNotRunnableException(StringUtils.format( + "Aborting request because task [%s] is not runnable", + taskId + )); + } + + String host = location.getHost(); + String scheme = ""; + int port = -1; + + try { + location = taskInfoProvider.getTaskLocation(taskId); + if (location.equals(TaskLocation.unknown())) { + throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", taskId)); + } + + host = location.getHost(); + scheme = location.getTlsPort() >= 0 ? "https" : "http"; + port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort(); + + // Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently + // for tasks that happen to still be starting up, so test the connection first to keep the logs clean. + checkConnection(host, port); + + try { + URI serviceUri = new URI( + scheme, + null, + host, + port, + path, + query, + null + ); + request = new Request(method, serviceUri.toURL()); + + // used to validate that we are talking to the correct worker + request.addHeader(ChatHandlerResource.TASK_ID_HEADER, taskId); + + if (content.length > 0) { + request.setContent(Preconditions.checkNotNull(mediaType, "mediaType"), content); + } + + log.debug("HTTP %s: %s", method.getName(), serviceUri.toString()); + response = httpClient.go(request, new FullResponseHandler(Charsets.UTF_8), httpTimeout).get(); + } + catch (Exception e) { + Throwables.propagateIfInstanceOf(e.getCause(), IOException.class); + Throwables.propagateIfInstanceOf(e.getCause(), ChannelException.class); + throw new RuntimeException(e); + } + + int responseCode = response.getStatus().getCode(); + if (responseCode / 100 == 2) { + return response; + } else if (responseCode == 400) { // don't bother retrying if it's a bad request + throw new IAE("Received 400 Bad Request with body: %s", response.getContent()); + } else { + throw new IOE("Received status [%d]", responseCode); + } + } + catch (IOException | ChannelException e) { + + // Since workers are free to move tasks around to different ports, there is a chance that a task may have been + // moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header + // identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the + // worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then + // we will wait for a short period then retry the request indefinitely, expecting the task's location to + // eventually be updated. + + final Duration delay; + if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) { + String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER); + if (headerId != null && !headerId.equals(taskId)) { + log.warn( + "Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s", + taskId, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS + ); + delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS); + } else { + delay = retryPolicy.getAndIncrementRetryDelay(); + } + } else { + delay = retryPolicy.getAndIncrementRetryDelay(); + } + String urlForLog = (request != null + ? request.getUrl().toString() + : StringUtils.format( + "%s://%s:%d%s", + scheme, + host, + port, + path + )); + if (!retry) { + // if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was + // for informational purposes only) so don't log a scary stack trace + log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage()); + throw e; + } else if (delay == null) { + log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog); + throw e; + } else { + try { + final long sleepTime = delay.getMillis(); + log.debug( + "Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])", + (response != null ? response.getStatus().getCode() : "no response"), + urlForLog, + new Duration(sleepTime).toString(), + (response != null ? response.getContent() : e.getMessage()) + ); + Thread.sleep(sleepTime); + } + catch (InterruptedException e2) { + e.addSuppressed(e2); + throw e; + } + } + } + catch (NoTaskLocationException e) { + log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or " + + "may have already completed", taskId); + throw e; + } + catch (Exception e) { + log.warn(e, "Exception while sending request"); + throw e; + } + } + } + + @Override + public void close() + { + executorService.shutdownNow(); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java index 792bc976326e..b0249440c7a1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskStatus.java @@ -24,11 +24,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; -import io.druid.indexer.TaskReport; import io.druid.indexer.TaskState; -import javax.annotation.Nullable; - /** * Represents the status of a task from the perspective of the coordinator. The task may be ongoing * ({@link #isComplete()} false) or it may be complete ({@link #isComplete()} true). @@ -39,45 +36,37 @@ public class TaskStatus { public static TaskStatus running(String taskId) { - return new TaskStatus(taskId, TaskState.RUNNING, null, -1); + return new TaskStatus(taskId, TaskState.RUNNING, -1); } public static TaskStatus success(String taskId) { - return new TaskStatus(taskId, TaskState.SUCCESS, null, -1); - } - - public static TaskStatus success(String taskId, TaskReport report) - { - return new TaskStatus(taskId, TaskState.SUCCESS, report, -1); + return new TaskStatus(taskId, TaskState.SUCCESS, -1); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, TaskState.FAILED, null, -1); + return new TaskStatus(taskId, TaskState.FAILED, -1); } public static TaskStatus fromCode(String taskId, TaskState code) { - return new TaskStatus(taskId, code, null, -1); + return new TaskStatus(taskId, code, -1); } private final String id; private final TaskState status; - private final TaskReport report; private final long duration; @JsonCreator protected TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") TaskState status, - @JsonProperty("report") @Nullable TaskReport report, @JsonProperty("duration") long duration ) { this.id = id; this.status = status; - this.report = report; this.duration = duration; // Check class invariants. @@ -97,12 +86,6 @@ public TaskState getStatusCode() return status; } - @JsonProperty("report") - public TaskReport getReport() - { - return report; - } - @JsonProperty("duration") public long getDuration() { @@ -158,7 +141,7 @@ public boolean isFailure() public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, report, _duration); + return new TaskStatus(id, status, _duration); } @Override @@ -173,14 +156,13 @@ public boolean equals(Object o) TaskStatus that = (TaskStatus) o; return duration == that.duration && java.util.Objects.equals(id, that.id) && - status == that.status && - java.util.Objects.equals(report, that.report); + status == that.status; } @Override public int hashCode() { - return java.util.Objects.hash(id, status, report, duration); + return java.util.Objects.hash(id, status, duration); } @Override @@ -189,7 +171,6 @@ public String toString() return Objects.toStringHelper(this) .add("id", id) .add("status", status) - .add("report", report) .add("duration", duration) .toString(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java new file mode 100644 index 000000000000..28ef838915b8 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java @@ -0,0 +1,55 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.google.common.base.Optional; +import com.google.inject.Inject; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.TaskStatusResponse; +import io.druid.indexer.TaskLocation; +import io.druid.indexing.common.TaskInfoProvider; +import io.druid.indexing.common.TaskStatus; + +public class ClientBasedTaskInfoProvider implements TaskInfoProvider +{ + private final IndexingServiceClient client; + + @Inject + public ClientBasedTaskInfoProvider(IndexingServiceClient client) + { + this.client = client; + } + + @Override + public TaskLocation getTaskLocation(String id) + { + final TaskStatusResponse response = client.getTaskStatus(id); + return response == null ? TaskLocation.unknown() : response.getStatus().getLocation(); + } + + @Override + public Optional getTaskStatus(String id) + { + final TaskStatusResponse response = client.getTaskStatus(id); + return response == null ? + Optional.absent() : + Optional.of(TaskStatus.fromCode(id, response.getStatus().getState())); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java new file mode 100644 index 000000000000..85bf33058e19 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java @@ -0,0 +1,29 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import io.druid.indexing.common.IndexTaskClient; +import io.druid.indexing.common.TaskInfoProvider; +import org.joda.time.Duration; + +public interface IndexTaskClientFactory +{ + T build(TaskInfoProvider taskInfoProvider, String callerId, int numThreads, Duration httpTimeout, long numRetries); +} diff --git a/api/src/main/java/io/druid/indexer/TaskReport.java b/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java similarity index 66% rename from api/src/main/java/io/druid/indexer/TaskReport.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java index 53dea93d97a7..f5a209381a4c 100644 --- a/api/src/main/java/io/druid/indexer/TaskReport.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java @@ -17,29 +17,28 @@ * under the License. */ -package io.druid.indexer; +package io.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.druid.timeline.DataSegment; -/** - * TaskReport can be optionally included in io.druid.indexing.common.TaskStatus to report some ingestion results to - * Supervisors or supervisorTasks. See ParallelIndexSinglePhaseSupervisorTask and ParallelIndexSinglePhaseSubTask - * as an example. - */ -public class TaskReport +import java.util.List; + +public class PushedSegmentsReport { private final String taskId; - private final Object payload; // can't use generic to not change TaskStatus + private final List segments; @JsonCreator - public TaskReport( + public PushedSegmentsReport( @JsonProperty("taskId") String taskId, - @JsonProperty("payload") Object payload + @JsonProperty("segments") List segments ) { - this.taskId = taskId; - this.payload = payload; + this.taskId = Preconditions.checkNotNull(taskId, "taskId"); + this.segments = Preconditions.checkNotNull(segments, "segments"); } @JsonProperty @@ -49,8 +48,8 @@ public String getTaskId() } @JsonProperty - public Object getPayload() + public List getSegments() { - return payload; + return segments; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIOConfig.java similarity index 90% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIOConfig.java index 28e122c62b0c..9ca0686fd5c2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIOConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIOConfig.java @@ -29,10 +29,10 @@ /** * Same with {@link IndexIOConfig} except its JSON type name. */ -@JsonTypeName("parallelIndexSinglePhase") -public class ParallelIndexSinglePhaseIOConfig extends IndexIOConfig +@JsonTypeName("index_single_phase_parallel") +public class SinglePhaseParallelIndexIOConfig extends IndexIOConfig { - public ParallelIndexSinglePhaseIOConfig( + public SinglePhaseParallelIndexIOConfig( @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting ) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIngestionSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIngestionSpec.java similarity index 69% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIngestionSpec.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIngestionSpec.java index c4f363852b37..c17824213369 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseIngestionSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIngestionSpec.java @@ -24,25 +24,25 @@ import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.IngestionSpec; -public class ParallelIndexSinglePhaseIngestionSpec - extends IngestionSpec +public class SinglePhaseParallelIndexIngestionSpec + extends IngestionSpec { private final DataSchema dataSchema; - private final ParallelIndexSinglePhaseIOConfig ioConfig; - private final ParallelIndexSinglePhaseTuningConfig tuningConfig; + private final SinglePhaseParallelIndexIOConfig ioConfig; + private final SinglePhaseParallelIndexTuningConfig tuningConfig; @JsonCreator - public ParallelIndexSinglePhaseIngestionSpec( + public SinglePhaseParallelIndexIngestionSpec( @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") ParallelIndexSinglePhaseIOConfig ioConfig, - @JsonProperty("tuningConfig") ParallelIndexSinglePhaseTuningConfig tuningConfig + @JsonProperty("ioConfig") SinglePhaseParallelIndexIOConfig ioConfig, + @JsonProperty("tuningConfig") SinglePhaseParallelIndexTuningConfig tuningConfig ) { super(dataSchema, ioConfig, tuningConfig); this.dataSchema = dataSchema; this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig == null ? ParallelIndexSinglePhaseTuningConfig.defaultConfig() : tuningConfig; + this.tuningConfig = tuningConfig == null ? SinglePhaseParallelIndexTuningConfig.defaultConfig() : tuningConfig; } @Override @@ -54,14 +54,14 @@ public DataSchema getDataSchema() @Override @JsonProperty("ioConfig") - public ParallelIndexSinglePhaseIOConfig getIOConfig() + public SinglePhaseParallelIndexIOConfig getIOConfig() { return ioConfig; } @Override @JsonProperty("tuningConfig") - public ParallelIndexSinglePhaseTuningConfig getTuningConfig() + public SinglePhaseParallelIndexTuningConfig getTuningConfig() { return tuningConfig; } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java similarity index 91% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java index f2a2f7ba07fd..bc9335c11b2c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; @@ -26,10 +27,10 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.druid.client.indexing.IndexingServiceClient; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.indexer.TaskReport; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.appenderator.CountingActionBasedSegmentAllocator; @@ -86,29 +87,33 @@ import java.util.stream.Collectors; /** - * A worker task of {@link ParallelIndexSinglePhaseSupervisorTask}. Similar to {@link IndexTask}, but this class returns - * a {@link TaskReport} to {@link ParallelIndexSinglePhaseSupervisorTask} including the information of pushed segments - * once it finishes its work. + * A worker task of {@link SinglePhaseParallelIndexSupervisorTask}. Similar to {@link IndexTask}, but this task + * generates and pushes segments, and reports them to the {@link SinglePhaseParallelIndexSupervisorTask} instead of + * publishing on its own. */ -public class ParallelIndexSinglePhaseSubTask extends AbstractTask +public class SinglePhaseParallelIndexSubTask extends AbstractTask { - private static final Logger log = new Logger(ParallelIndexSinglePhaseSubTask.class); - private static final String TYPE = "parallelIndexSinglePhaseSubIndex"; + private static final Logger log = new Logger(SinglePhaseParallelIndexSubTask.class); + private static final String TYPE = "index_single_phase_sub"; private final int numAttempts; - private final IndexTask.IndexIngestionSpec ingestionSchema; + private final SinglePhaseParallelIndexIngestionSpec ingestionSchema; private final String supervisorTaskId; + private final IndexingServiceClient indexingServiceClient; + private final IndexTaskClientFactory taskClientFactory; @JsonCreator - public ParallelIndexSinglePhaseSubTask( - // id shouldn't be null except when this task is created by ParallelIndexSinglePhaseSupervisorTask + public SinglePhaseParallelIndexSubTask( + // id shouldn't be null except when this task is created by SinglePhaseParallelIndexSupervisorTask @JsonProperty("id") @Nullable final String id, @JsonProperty("groupId") final String groupId, @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("supervisorTaskId") final String supervisorTaskId, @JsonProperty("numAttempts") final int numAttempts, // zero-based counting - @JsonProperty("spec") final IndexTask.IndexIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context + @JsonProperty("spec") final SinglePhaseParallelIndexIngestionSpec ingestionSchema, + @JsonProperty("context") final Map context, + @JacksonInject IndexingServiceClient indexingServiceClient, + @JacksonInject IndexTaskClientFactory taskClientFactory ) { super( @@ -119,13 +124,15 @@ public ParallelIndexSinglePhaseSubTask( context ); - this.numAttempts = numAttempts; - this.ingestionSchema = ingestionSchema; - this.supervisorTaskId = supervisorTaskId; - if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { throw new UnsupportedOperationException("Guaranteed rollup is not supported"); } + + this.numAttempts = numAttempts; + this.ingestionSchema = ingestionSchema; + this.supervisorTaskId = supervisorTaskId; + this.indexingServiceClient = indexingServiceClient; + this.taskClientFactory = taskClientFactory; } @Override @@ -169,7 +176,7 @@ public int getNumAttempts() } @JsonProperty("spec") - public IndexTask.IndexIngestionSpec getIngestionSchema() + public SinglePhaseParallelIndexIngestionSpec getIngestionSchema() { return ingestionSchema; } @@ -234,8 +241,16 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception firehoseFactory, firehoseTempDir ); + final SinglePhaseParallelIndexTaskClient taskClient = taskClientFactory.build( + new ClientBasedTaskInfoProvider(indexingServiceClient), + getId(), + 1, // always use a single http thread + ingestionSchema.getTuningConfig().getChatHandlerTimeout(), + ingestionSchema.getTuningConfig().getChatHandlerNumRetries() + ); + taskClient.report(supervisorTaskId, pushedSegments); - return TaskStatus.success(getId(), new TaskReport(getId(), pushedSegments)); + return TaskStatus.success(getId()); } private Map tryAcquireExclusiveSurrogateLocks( @@ -314,7 +329,7 @@ private static IndexTask.ShardSpecs createShardSpecWithoutInputScan(GranularityS } private static IndexTask.ShardSpecs createShardSpecsFromInput( - IndexTask.IndexIngestionSpec ingestionSchema, + SinglePhaseParallelIndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, File firehoseTempDir, GranularitySpec granularitySpec, @@ -341,7 +356,7 @@ private static IndexTask.ShardSpecs createShardSpecsFromInput( } private static List collectIntervalsAndShardSpecs( - IndexTask.IndexIngestionSpec ingestionSchema, + SinglePhaseParallelIndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, File firehoseTempDir, GranularitySpec granularitySpec, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java similarity index 72% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java index 6a10deb2cf4e..26fbf3fae880 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java @@ -19,19 +19,17 @@ package io.druid.indexing.common.task; -import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; - import java.util.Map; -final class ParallelIndexSinglePhaseSubTaskSpec extends SubTaskSpec +class SinglePhaseParallelIndexSubTaskSpec extends SubTaskSpec { - private final IndexIngestionSpec ingestionSpec; + private final SinglePhaseParallelIndexIngestionSpec ingestionSpec; - ParallelIndexSinglePhaseSubTaskSpec( + SinglePhaseParallelIndexSubTaskSpec( String id, String groupId, String supervisorTaskId, - IndexIngestionSpec indexIngestionSpec, + SinglePhaseParallelIndexIngestionSpec indexIngestionSpec, Map context ) { @@ -39,22 +37,24 @@ final class ParallelIndexSinglePhaseSubTaskSpec extends SubTaskSpec baseFirehoseFactory; private final int maxNumTasks; - private final TaskMonitor taskMonitor; + private final IndexingServiceClient indexingServiceClient; + private final ChatHandlerProvider chatHandlerProvider; + private final AuthorizerMapper authorizerMapper; - private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); - private final List segments = new ArrayList<>(); + private final BlockingQueue> taskCompleteEvents = + new LinkedBlockingDeque<>(); + + // subtaskId -> report + private final ConcurrentMap segmentsMap = new ConcurrentHashMap<>(); private volatile boolean stopped; private int nextSpecId = 0; @JsonCreator - public ParallelIndexSinglePhaseSupervisorTask( + public SinglePhaseParallelIndexSupervisorTask( @JsonProperty("id") String id, @JsonProperty("resource") TaskResource taskResource, - @JsonProperty("spec") ParallelIndexSinglePhaseIngestionSpec ingestionSchema, + @JsonProperty("spec") SinglePhaseParallelIndexIngestionSpec ingestionSchema, @JsonProperty("context") Map context, - @JacksonInject IndexingServiceClient indexingServiceClient + @JacksonInject @Nullable IndexingServiceClient indexingServiceClient, // null in overlords + @JacksonInject @Nullable ChatHandlerProvider chatHandlerProvider, // null in overlords + @JacksonInject AuthorizerMapper authorizerMapper ) { super( @@ -118,7 +138,9 @@ public ParallelIndexSinglePhaseSupervisorTask( this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumBatchTasks(); - this.taskMonitor = new TaskMonitor<>(indexingServiceClient, ingestionSchema.getTuningConfig().getMaxRetry()); + this.indexingServiceClient = indexingServiceClient; + this.chatHandlerProvider = chatHandlerProvider; + this.authorizerMapper = authorizerMapper; } @Override @@ -134,7 +156,7 @@ public String getType() } @JsonProperty("spec") - ParallelIndexSinglePhaseIngestionSpec getIngestionSchema() + public SinglePhaseParallelIndexIngestionSpec getIngestionSchema() { return ingestionSchema; } @@ -165,7 +187,7 @@ static boolean isReady(TaskActionClient actionClient, SortedSet interv } @Override - public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { if (baseFirehoseFactory.isSplittable()) { return runParallel(toolbox); @@ -178,9 +200,20 @@ public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Excep } } - private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) throws Exception + private TaskStatus runParallel(TaskToolbox toolbox) throws Exception { - final Iterator subTaskSpecIterator = subTaskSpecIterator(); + final TaskMonitor taskMonitor = new TaskMonitor<>( + Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"), + ingestionSchema.getTuningConfig().getMaxRetry() + ); + + log.info( + "Found chat handler of class[%s]", + Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName() + ); + chatHandlerProvider.register(getId(), this, false); + + final Iterator subTaskSpecIterator = subTaskSpecIterator(); final int numTotalTasks = baseFirehoseFactory.getNumSplits(); final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs(); TaskState state = TaskState.FAILED; @@ -194,12 +227,12 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr log.info("Submitting initial tasks"); // Submit initial tasks while (subTaskSpecIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) { - submitNewTask(subTaskSpecIterator.next()); + submitNewTask(taskMonitor, subTaskSpecIterator.next()); } log.info("Waiting for subTasks to be completed"); while (!stopped && !Thread.currentThread().isInterrupted()) { - final SubTaskCompleteEvent taskCompleteEvent = taskCompleteEvents.poll( + final SubTaskCompleteEvent taskCompleteEvent = taskCompleteEvents.poll( taskStatusCheckingPeriod, TimeUnit.MILLISECONDS ); @@ -209,7 +242,15 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr switch (completeState) { case SUCCESS: numCompleteTasks++; - segments.addAll((Collection) taskCompleteEvent.getLastStatus().getReport().getPayload()); + final TaskStatusPlus completeStatus = taskCompleteEvent.getLastStatus(); + if (completeStatus == null) { + throw new ISE("Last status of complete task is missing!"); + } + // Pushed segments of complete tasks are supposed to be already reported. + if (!segmentsMap.containsKey(completeStatus.getId())) { + throw new ISE("Missing reports from task[%s]!", completeStatus.getId()); + } + log.info("[%d/%d] tasks succeeded", numCompleteTasks, numTotalTasks); if (!subTaskSpecIterator.hasNext()) { if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) { @@ -230,19 +271,19 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr } } } else if (taskMonitor.getNumRunningTasks() < maxNumTasks) { - submitNewTask(subTaskSpecIterator.next()); + submitNewTask(taskMonitor, subTaskSpecIterator.next()); } break; case FAILED: // TaskMonitor already tried everything it can do for failed tasks. We failed. state = TaskState.FAILED; stopped = true; - final TaskStatus lastStatus = taskCompleteEvent.getLastStatus(); + final TaskStatusPlus lastStatus = taskCompleteEvent.getLastStatus(); if (lastStatus != null) { log.error("Failed because of the failed sub task[%s]", lastStatus.getId()); } else { - final ParallelIndexSinglePhaseSubTaskSpec spec = - (ParallelIndexSinglePhaseSubTaskSpec) taskCompleteEvent.getSpec(); + final SinglePhaseParallelIndexSubTaskSpec spec = + (SinglePhaseParallelIndexSubTaskSpec) taskCompleteEvent.getSpec(); log.error( "Failed to run sub tasks for inputSplits[%s]", getSplitsIfSplittable(spec.getIngestionSpec().getIOConfig().getFirehoseFactory()) @@ -260,6 +301,7 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr // Cleanup resources taskCompleteEvents.clear(); taskMonitor.stop(); + chatHandlerProvider.unregister(getId()); if (state != TaskState.SUCCESS) { log.info( @@ -274,10 +316,10 @@ private io.druid.indexing.common.TaskStatus runParallel(TaskToolbox toolbox) thr } } - return io.druid.indexing.common.TaskStatus.fromCode(getId(), state); + return TaskStatus.fromCode(getId(), state); } - private io.druid.indexing.common.TaskStatus runSequential(TaskToolbox toolbox) throws Exception + private TaskStatus runSequential(TaskToolbox toolbox) throws Exception { return new IndexTask( getId(), @@ -293,7 +335,28 @@ private io.druid.indexing.common.TaskStatus runSequential(TaskToolbox toolbox) t ).run(toolbox); } - private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexSinglePhaseTuningConfig tuningConfig) + @POST + @Path("/report") + @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE) + public Response report( + PushedSegmentsReport report, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck(req, Action.WRITE, getDataSource(), authorizerMapper); + collectReport(report); + return Response.status(Response.Status.OK).build(); + } + + @VisibleForTesting + void collectReport(PushedSegmentsReport report) + { + if (segmentsMap.put(report.getTaskId(), report) != null) { + throw new ISE("Dupliate task report from task[%s]", report.getTaskId()); + } + } + + private static IndexTuningConfig convertToIndexTuningConfig(SinglePhaseParallelIndexTuningConfig tuningConfig) { return new IndexTuningConfig( tuningConfig.getTargetPartitionSize(), @@ -320,15 +383,21 @@ private void publish(TaskToolbox toolbox) throws IOException return toolbox.getTaskActionClient().submit(action).isSuccess(); }; final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()); - final Set segmentsToPublish = ImmutableSet.copyOf(segments); + final Set segmentsToPublish = segmentsMap + .values() + .stream() + .flatMap(report -> report.getSegments().stream()) + .collect(Collectors.toSet()); final boolean published = publisher.publishSegments(segmentsToPublish, null); if (published) { log.info("Published segments"); } else { log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); - final Set segmentsIdentifiers = segments + final Set segmentsIdentifiers = segmentsMap + .values() .stream() + .flatMap(report -> report.getSegments().stream()) .map(SegmentIdentifier::fromDataSegment) .collect(Collectors.toSet()); if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers) @@ -340,15 +409,18 @@ private void publish(TaskToolbox toolbox) throws IOException } } - private void submitNewTask(ParallelIndexSinglePhaseSubTaskSpec spec) + private void submitNewTask( + TaskMonitor taskMonitor, + SinglePhaseParallelIndexSubTaskSpec spec + ) { - final ListenableFuture> future = taskMonitor.submit(spec); + final ListenableFuture> future = taskMonitor.submit(spec); Futures.addCallback( future, - new FutureCallback>() + new FutureCallback>() { @Override - public void onSuccess(SubTaskCompleteEvent completeEvent) + public void onSuccess(SubTaskCompleteEvent completeEvent) { // this callback is called if a task completed wheter it succeeded or not. taskCompleteEvents.offer(completeEvent); @@ -375,25 +447,31 @@ public void onFailure(Throwable t) } @VisibleForTesting - Iterator subTaskSpecIterator() throws IOException + int getAndIncreaseNextSpecId() + { + return nextSpecId++; + } + + @VisibleForTesting + Iterator subTaskSpecIterator() throws IOException { return Iterators.transform(baseFirehoseFactory.getSplits(), this::newTaskSpec); } @VisibleForTesting - ParallelIndexSinglePhaseSubTaskSpec newTaskSpec(InputSplit split) + SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) { - return new ParallelIndexSinglePhaseSubTaskSpec( - getId() + "_" + nextSpecId++, + return new SinglePhaseParallelIndexSubTaskSpec( + getId() + "_" + getAndIncreaseNextSpecId(), getGroupId(), getId(), - new IndexIngestionSpec( + new SinglePhaseParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), - new IndexIOConfig( + new SinglePhaseParallelIndexIOConfig( baseFirehoseFactory.withSplit(split), ingestionSchema.getIOConfig().isAppendToExisting() ), - convertToIndexTuningConfig(ingestionSchema.getTuningConfig()) + ingestionSchema.getTuningConfig() ), getContext() ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java new file mode 100644 index 000000000000..4084b634a5dd --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java @@ -0,0 +1,80 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.indexing.common.IndexTaskClient; +import io.druid.indexing.common.TaskInfoProvider; +import io.druid.java.util.common.ISE; +import io.druid.java.util.http.client.HttpClient; +import io.druid.java.util.http.client.response.FullResponseHolder; +import io.druid.timeline.DataSegment; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.Duration; + +import java.io.IOException; +import java.util.List; + +public class SinglePhaseParallelIndexTaskClient extends IndexTaskClient +{ + private final String subtaskId; + + public SinglePhaseParallelIndexTaskClient( + HttpClient httpClient, + ObjectMapper objectMapper, + TaskInfoProvider taskInfoProvider, + Duration httpTimeout, + String callerId, + long numRetries + ) + { + super(httpClient, objectMapper, taskInfoProvider, httpTimeout, callerId, 1, numRetries); + this.subtaskId = callerId; + } + + public String getSubtaskId() + { + return subtaskId; + } + + public void report(String supervisorTaskId, List pushedSegments) + { + try { + final FullResponseHolder response = submitSmilRequest( + supervisorTaskId, + HttpMethod.POST, + "report", + null, + serialize(new PushedSegmentsReport(subtaskId, pushedSegments)), + true + ); + if (!isSuccess(response)) { + throw new ISE( + "Failed to send taskReports to task[%s] with the HTTP code [%d]", + supervisorTaskId, + response.getStatus().getCode() + ); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClientFactory.java new file mode 100644 index 000000000000..be6679988710 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClientFactory.java @@ -0,0 +1,66 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import io.druid.guice.annotations.EscalatedGlobal; +import io.druid.guice.annotations.Smile; +import io.druid.indexing.common.TaskInfoProvider; +import io.druid.java.util.http.client.HttpClient; +import org.joda.time.Duration; + +public class SinglePhaseParallelIndexTaskClientFactory + implements IndexTaskClientFactory +{ + private final HttpClient httpClient; + private final ObjectMapper mapper; + + @Inject + public SinglePhaseParallelIndexTaskClientFactory( + @EscalatedGlobal HttpClient httpClient, + @Smile ObjectMapper mapper + ) + { + this.httpClient = httpClient; + this.mapper = mapper; + } + + @Override + public SinglePhaseParallelIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String callerId, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + Preconditions.checkState(numThreads == 1, "expect numThreads to be 1"); + return new SinglePhaseParallelIndexTaskClient( + httpClient, + mapper, + taskInfoProvider, + httpTimeout, + callerId, + numRetries + ); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java similarity index 78% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java index 810aad1574c4..076dacd456cc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java @@ -25,24 +25,34 @@ import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; import io.druid.segment.IndexSpec; import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Duration; +import org.joda.time.Period; import javax.annotation.Nullable; import java.util.Objects; -@JsonTypeName("parallelIndexSinglePhase") -public class ParallelIndexSinglePhaseTuningConfig extends IndexTuningConfig +@JsonTypeName("index_single_phase_parallel") +public class SinglePhaseParallelIndexTuningConfig extends IndexTuningConfig { private static final int DEFAULT_MAX_NUM_BATCH_TASKS = Integer.MAX_VALUE; // unlimited private static final int DEFAULT_MAX_RETRY = 3; private static final long DEFAULT_TASK_STATUS_CHECK_PERIOD_MS = 1000; + private static final Duration DEFAULT_CHAT_HANDLER_TIMEOUT = new Period("PT10S").toStandardDuration(); + private static final int DEFAULT_CHAT_HANDLER_NUM_RETRIES = 5; + private final int maxNumBatchTasks; private final int maxRetry; private final long taskStatusCheckPeriodMs; - public static ParallelIndexSinglePhaseTuningConfig defaultConfig() + private final Duration chatHandlerTimeout; + private final int chatHandlerNumRetries; + + public static SinglePhaseParallelIndexTuningConfig defaultConfig() { - return new ParallelIndexSinglePhaseTuningConfig( + return new SinglePhaseParallelIndexTuningConfig( + null, + null, null, null, null, @@ -60,7 +70,7 @@ public static ParallelIndexSinglePhaseTuningConfig defaultConfig() } @JsonCreator - public ParallelIndexSinglePhaseTuningConfig( + public SinglePhaseParallelIndexTuningConfig( @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, @@ -73,7 +83,9 @@ public ParallelIndexSinglePhaseTuningConfig( @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("maxNumBatchTasks") @Nullable Integer maxNumBatchTasks, @JsonProperty("maxRetry") @Nullable Integer maxRetry, - @JsonProperty("taskStatusCheckPeriodMs") @Nullable Integer taskStatusCheckPeriodMs + @JsonProperty("taskStatusCheckPeriodMs") @Nullable Integer taskStatusCheckPeriodMs, + @JsonProperty("chatHandlerTimeout") @Nullable Duration chatHandlerTimeout, + @JsonProperty("chatHandlerNumRetries") @Nullable Integer chatHandlerNumRetries ) { super( @@ -86,7 +98,7 @@ public ParallelIndexSinglePhaseTuningConfig( maxPendingPersists, null, forceExtendableShardSpecs, - false, // ParallelIndexSinglePhaseSupervisorTask can't be used for guaranteed rollup + false, // SinglePhaseParallelIndexSupervisorTask can't be used for guaranteed rollup reportParseExceptions, null, pushTimeout, @@ -98,6 +110,9 @@ public ParallelIndexSinglePhaseTuningConfig( this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs == null ? DEFAULT_TASK_STATUS_CHECK_PERIOD_MS : taskStatusCheckPeriodMs; + + this.chatHandlerTimeout = DEFAULT_CHAT_HANDLER_TIMEOUT; + this.chatHandlerNumRetries = DEFAULT_CHAT_HANDLER_NUM_RETRIES; } @JsonProperty @@ -118,6 +133,18 @@ public long getTaskStatusCheckPeriodMs() return taskStatusCheckPeriodMs; } + @JsonProperty + public Duration getChatHandlerTimeout() + { + return chatHandlerTimeout; + } + + @JsonProperty + public int getChatHandlerNumRetries() + { + return chatHandlerNumRetries; + } + @Override public boolean equals(Object o) { @@ -127,7 +154,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ParallelIndexSinglePhaseTuningConfig that = (ParallelIndexSinglePhaseTuningConfig) o; + SinglePhaseParallelIndexTuningConfig that = (SinglePhaseParallelIndexTuningConfig) o; return getMaxRowsInMemory() == that.getMaxRowsInMemory() && Objects.equals(getMaxTotalRows(), that.getMaxTotalRows()) && getMaxPendingPersists() == that.getMaxPendingPersists() && @@ -141,7 +168,9 @@ public boolean equals(Object o) Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()) && maxNumBatchTasks == that.maxNumBatchTasks && maxRetry == that.maxRetry && - taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs; + taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs && + chatHandlerTimeout.equals(that.chatHandlerTimeout) && + chatHandlerNumRetries == that.chatHandlerNumRetries; } @Override @@ -161,7 +190,9 @@ public int hashCode() getSegmentWriteOutMediumFactory(), maxNumBatchTasks, maxRetry, - taskStatusCheckPeriodMs + taskStatusCheckPeriodMs, + chatHandlerTimeout, + chatHandlerNumRetries ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index d8e3d2e61fdb..c4f9fad805da 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -51,8 +51,8 @@ @JsonSubTypes.Type(name = "archive", value = ArchiveTask.class), @JsonSubTypes.Type(name = "restore", value = RestoreTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), - @JsonSubTypes.Type(name = "parallelIndexSinglePhase", value = ParallelIndexSinglePhaseSupervisorTask.class), - @JsonSubTypes.Type(name = "subIndexSinglePhase", value = ParallelIndexSinglePhaseSubTask.class), + @JsonSubTypes.Type(name = "index_single_phase_parallel", value = SinglePhaseParallelIndexSupervisorTask.class), + @JsonSubTypes.Type(name = "index_single_phase_sub", value = SinglePhaseParallelIndexSubTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment", value = HadoopConverterTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment_sub", value = HadoopConverterTask.ConverterSubTask.class), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index f096b6cc3a0b..76dd08c13baa 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -24,9 +24,9 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import io.druid.client.indexing.IndexingServiceClient; -import io.druid.client.indexing.TaskStatus; import io.druid.client.indexing.TaskStatusResponse; import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; @@ -85,8 +85,8 @@ public void start(long taskStatusCheckingPeriod) final MonitorEntry monitorEntry = entry.getValue(); final TaskStatusResponse taskStatusResponse = indexingServiceClient.getTaskStatus(taskId); if (taskStatusResponse != null) { - final TaskStatus taskStatus = taskStatusResponse.getStatus(); - switch (taskStatus.getStatusCode()) { + final TaskStatusPlus taskStatus = taskStatusResponse.getStatus(); + switch (taskStatus.getState()) { case SUCCESS: numRunningTasks.decrementAndGet(); iterator.remove(); @@ -100,10 +100,9 @@ public void start(long taskStatusCheckingPeriod) log.warn("task[%s] failed!", taskId); if (monitorEntry.numTries() < maxRetry) { log.info( - "We still have chnaces[%d/%d] to complete. Retrying spec[%s]", + "We still have chnaces[%d/%d] to complete.", monitorEntry.numTries(), - maxRetry, - monitorEntry.spec.getId() + maxRetry ); retry(monitorEntry, taskStatus); } else { @@ -119,7 +118,7 @@ public void start(long taskStatusCheckingPeriod) // do nothing break; default: - throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getStatusCode(), taskId); + throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getState(), taskId); } } } @@ -157,11 +156,12 @@ public ListenableFuture> submit(SubTaskSpec spec) return taskFuture; } - private void retry(MonitorEntry monitorEntry, TaskStatus lastFailedTaskStatus) + private void retry(MonitorEntry monitorEntry, TaskStatusPlus lastFailedTaskStatus) { if (running) { final SubTaskSpec spec = monitorEntry.spec; final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1); + log.info("Submitting a new task[%s] for retrying spec[%s]", task.getId(), spec.getId()); indexingServiceClient.runTask(task); numRunningTasks.incrementAndGet(); @@ -190,7 +190,7 @@ private class MonitorEntry { private final SubTaskSpec spec; private final T runningTask; - private final List taskHistory; + private final List taskHistory; private final SettableFuture> completeEventFuture; MonitorEntry( @@ -205,7 +205,7 @@ private class MonitorEntry private MonitorEntry( SubTaskSpec spec, T runningTask, - List taskHistory, + List taskHistory, SettableFuture> completeEventFuture ) { @@ -215,7 +215,7 @@ private MonitorEntry( this.completeEventFuture = completeEventFuture; } - MonitorEntry withNewRunningTask(T newTask, TaskStatus statusOfLastTask) + MonitorEntry withNewRunningTask(T newTask, TaskStatusPlus statusOfLastTask) { taskHistory.add(statusOfLastTask); return new MonitorEntry( @@ -231,7 +231,7 @@ int numTries() return taskHistory.size() + 1; // count runningTask. this is valid only until setLastStatus() is called } - void setLastStatus(TaskStatus lastStatus) + void setLastStatus(TaskStatusPlus lastStatus) { if (!runningTask.getId().equals(lastStatus.getId())) { throw new ISE( @@ -242,7 +242,7 @@ void setLastStatus(TaskStatus lastStatus) } taskHistory.add(lastStatus); - completeEventFuture.set(new SubTaskCompleteEvent<>(spec, lastStatus.getStatusCode(), taskHistory)); + completeEventFuture.set(new SubTaskCompleteEvent<>(spec, lastStatus.getState(), taskHistory)); } } @@ -251,12 +251,12 @@ static class SubTaskCompleteEvent private final SubTaskSpec spec; private final TaskState lastState; @Nullable - private final List attemptHistory; + private final List attemptHistory; SubTaskCompleteEvent( SubTaskSpec spec, TaskState lastState, - @Nullable List attemptHistory + @Nullable List attemptHistory ) { this.spec = Preconditions.checkNotNull(spec, "spec"); @@ -275,13 +275,13 @@ TaskState getLastState() } @Nullable - List getAttemptHistory() + List getAttemptHistory() { return attemptHistory; } @Nullable - TaskStatus getLastStatus() + TaskStatusPlus getLastStatus() { return attemptHistory == null ? null : attemptHistory.get(attemptHistory.size() - 1); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 8be1dcdbe95e..dfede22a4c62 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -237,16 +237,63 @@ public Response getTaskPayload(@PathParam("taskid") String taskid) @ResourceFilters(TaskResourceFilter.class) public Response getTaskStatus(@PathParam("taskid") String taskid) { - final TaskStatusResponse response = new TaskStatusResponse( - taskid, - taskStorageQueryAdapter.getStatus(taskid).orNull() - ); + if (taskMaster.getTaskRunner().isPresent()) { + final TaskRunner taskRunner = taskMaster.getTaskRunner().get(); - final Response.Status status = response.getStatus() == null - ? Response.Status.NOT_FOUND - : Response.Status.OK; + final Task task = taskStorageQueryAdapter.getTask(taskid).orNull(); + final TaskStatus taskStatus = taskStorageQueryAdapter.getStatus(taskid).orNull(); + final Pair createdDateAndDataSource = taskStorageQueryAdapter.getCreatedDateAndDataSource( + taskid + ); + final TaskStatusResponse response; + if (task != null && taskStatus != null && createdDateAndDataSource != null) { + final TaskRunnerWorkItem workItem = taskRunner + .getKnownTasks() + .stream() + .filter(item -> item.getTaskId().equals(taskid)) + .findAny() + .orElse(null); + if (workItem != null) { + response = new TaskStatusResponse( + taskid, + new TaskStatusPlus( + taskid, + task.getType(), + createdDateAndDataSource.lhs, + workItem.getQueueInsertionTime(), + taskStatus.getStatusCode(), + taskStatus.getDuration(), + workItem.getLocation(), + createdDateAndDataSource.rhs + ) + ); + } else { + response = new TaskStatusResponse( + taskid, + new TaskStatusPlus( + taskid, + task.getType(), + createdDateAndDataSource.lhs, + DateTimes.EPOCH, + taskStatus.getStatusCode(), + taskStatus.getDuration(), + TaskLocation.unknown(), + createdDateAndDataSource.rhs + ) + ); + } + } else { + response = new TaskStatusResponse(taskid, null); + } - return Response.status(status).entity(response).build(); + final Response.Status status = response.getStatus() == null + ? Response.Status.NOT_FOUND + : Response.Status.OK; + + return Response.status(status).entity(response).build(); + } else { + return Response.status(Response.Status.NOT_FOUND).entity("No tasks are running").build(); + } } @GET diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java index b15aeb0cc0a8..5e413ce92733 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java @@ -21,19 +21,19 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.indexing.common.TaskStatus; +import io.druid.indexer.TaskStatusPlus; import java.util.Objects; public class TaskStatusResponse { private final String task; // Task ID, named "task" in the JSONification of this class. - private final TaskStatus status; + private final TaskStatusPlus status; @JsonCreator public TaskStatusResponse( @JsonProperty("task") final String task, - @JsonProperty("status") final TaskStatus status + @JsonProperty("status") final TaskStatusPlus status ) { this.task = task; @@ -47,7 +47,7 @@ public String getTask() } @JsonProperty - public TaskStatus getStatus() + public TaskStatusPlus getStatus() { return status; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index ad8da3bfe121..76a7eb5ecb29 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -23,6 +23,9 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Stopwatch; +import com.google.common.collect.ImmutableMap; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.NoopIndexingServiceClient; import io.druid.guice.ServerModule; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; @@ -73,16 +76,17 @@ public int columnCacheSizeBytes() for (Module module : list) { jsonMapper.registerModule(module); } - jsonMapper.setInjectableValues( new InjectableValues.Std() - .addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE) + .addValue(ExprMacroTable.class, LookupEnabledTestExprMacroTable.INSTANCE) .addValue(IndexIO.class, indexIO) .addValue(ObjectMapper.class, jsonMapper) .addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider()) .addValue(AuthConfig.class, new AuthConfig()) .addValue(AuthorizerMapper.class, null) .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) + .addValue(IndexingServiceClient.class, new NoopIndexingServiceClient()) + .addValue(AuthorizerMapper.class, new AuthorizerMapper(ImmutableMap.of())) ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskSerdeTest.java new file mode 100644 index 000000000000..fd4f85eb49c0 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskSerdeTest.java @@ -0,0 +1,156 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.client.indexing.NoopIndexingServiceClient; +import io.druid.data.input.impl.CSVParseSpec; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.ParseSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.indexing.common.TestUtils; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.realtime.firehose.LocalFirehoseFactory; +import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import io.druid.server.security.AuthorizerMapper; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class SinglePhaseParallelIndexSupervisorTaskSerdeTest +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( + new TimestampSpec( + "ts", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("ts", "dim", "val"), + false, + 0 + ); + + private final TestUtils testUtils = new TestUtils(); + + @Test + public void testSerde() throws IOException + { + final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); + objectMapper.registerSubtypes( + new NamedType(LocalFirehoseFactory.class, "local") + ); + + final SinglePhaseParallelIndexSupervisorTask task = newTask( + objectMapper, + Intervals.of("2018/2019") + ); + final String json = objectMapper.writeValueAsString(task); + Assert.assertEquals(task, objectMapper.readValue(json, Task.class)); + } + + private SinglePhaseParallelIndexSupervisorTask newTask( + ObjectMapper objectMapper, + Interval interval + ) + { + // set up ingestion spec + final SinglePhaseParallelIndexIngestionSpec singlePhaseIngestionSpec = new SinglePhaseParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + objectMapper.convertValue( + new StringInputRowParser( + DEFAULT_PARSE_SPEC, + null + ), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null, + objectMapper + ), + new SinglePhaseParallelIndexIOConfig( + new LocalFirehoseFactory(new File("tmp"), "test_*", null), + false + ), + new SinglePhaseParallelIndexTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + 2, + null, + null, + null, + null + ) + ); + + // set up test tools + return new SinglePhaseParallelIndexSupervisorTask( + "taskId", + new TaskResource("group", 1), + singlePhaseIngestionSpec, + new HashMap<>(), + new NoopIndexingServiceClient(), + new NoopChatHandlerProvider(), + new AuthorizerMapper(ImmutableMap.of()) + ); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java similarity index 64% rename from indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java index c5b16ea04143..fc4a70269442 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSinglePhaseSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java @@ -20,11 +20,13 @@ package io.druid.indexing.common.task; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.NoopIndexingServiceClient; import io.druid.client.indexing.TaskStatusResponse; import io.druid.data.input.FiniteFirehoseFactory; import io.druid.data.input.InputSplit; @@ -33,10 +35,14 @@ import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; +import io.druid.indexer.TaskLocation; import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.TaskInfoProvider; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; @@ -49,6 +55,10 @@ import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; +import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import io.druid.server.security.AuthorizerMapper; +import io.druid.timeline.DataSegment; +import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -67,13 +77,14 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -public class ParallelIndexSinglePhaseSupervisorTaskTest extends IngestionTestBase +public class SinglePhaseParallelIndexSupervisorTaskTest extends IngestionTestBase { @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -134,9 +145,9 @@ public void teardown() @Test public void testIsReady() throws Exception { - final ParallelIndexSinglePhaseSupervisorTask task = newTask( + final SinglePhaseParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), - new ParallelIndexSinglePhaseIOConfig( + new SinglePhaseParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false ) @@ -147,18 +158,20 @@ public void testIsReady() throws Exception prepareTaskForLocking(task); Assert.assertTrue(task.isReady(actionClient)); - final Iterator subTaskSpecIterator = task.subTaskSpecIterator(); + final Iterator subTaskSpecIterator = task.subTaskSpecIterator(); while (subTaskSpecIterator.hasNext()) { - final ParallelIndexSinglePhaseSubTaskSpec spec = subTaskSpecIterator.next(); - final ParallelIndexSinglePhaseSubTask subTask = new ParallelIndexSinglePhaseSubTask( + final SinglePhaseParallelIndexSubTaskSpec spec = subTaskSpecIterator.next(); + final SinglePhaseParallelIndexSubTask subTask = new SinglePhaseParallelIndexSubTask( null, spec.getGroupId(), null, spec.getSupervisorTaskId(), 0, spec.getIngestionSpec(), - spec.getContext() + spec.getContext(), + indexingServiceClient, + null ); final TaskActionClient subTaskActionClient = createActionClient(subTask); prepareTaskForLocking(subTask); @@ -169,9 +182,9 @@ public void testIsReady() throws Exception @Test public void testWithoutInterval() throws Exception { - final ParallelIndexSinglePhaseSupervisorTask task = newTask( + final SinglePhaseParallelIndexSupervisorTask task = newTask( null, - new ParallelIndexSinglePhaseIOConfig( + new SinglePhaseParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false ) @@ -187,9 +200,9 @@ public void testWithoutInterval() throws Exception @Test() public void testRunInParallel() throws Exception { - final ParallelIndexSinglePhaseSupervisorTask task = newTask( + final SinglePhaseParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), - new ParallelIndexSinglePhaseIOConfig( + new SinglePhaseParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false ) @@ -205,9 +218,9 @@ public void testRunInParallel() throws Exception @Test public void testRunInSequential() throws Exception { - final ParallelIndexSinglePhaseSupervisorTask task = newTask( + final SinglePhaseParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), - new ParallelIndexSinglePhaseIOConfig( + new SinglePhaseParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null) { @Override @@ -227,13 +240,13 @@ public boolean isSplittable() Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); } - private ParallelIndexSinglePhaseSupervisorTask newTask( + private SinglePhaseParallelIndexSupervisorTask newTask( Interval interval, - ParallelIndexSinglePhaseIOConfig ioConfig + SinglePhaseParallelIndexIOConfig ioConfig ) { // set up ingestion spec - final ParallelIndexSinglePhaseIngestionSpec singlePhaseIngestionSpec = new ParallelIndexSinglePhaseIngestionSpec( + final SinglePhaseParallelIndexIngestionSpec singlePhaseIngestionSpec = new SinglePhaseParallelIndexIngestionSpec( new DataSchema( "dataSource", getObjectMapper().convertValue( @@ -255,7 +268,7 @@ private ParallelIndexSinglePhaseSupervisorTask newTask( getObjectMapper() ), ioConfig, - new ParallelIndexSinglePhaseTuningConfig( + new SinglePhaseParallelIndexTuningConfig( null, null, null, @@ -268,12 +281,14 @@ private ParallelIndexSinglePhaseSupervisorTask newTask( null, 2, null, + null, + null, null ) ); // set up test tools - return new TestParallelIndexSinglePhaseSupervisorTask( + return new TestSinglePhaseParallelIndexSupervisorTask( null, null, singlePhaseIngestionSpec, @@ -282,22 +297,29 @@ private ParallelIndexSinglePhaseSupervisorTask newTask( ); } - private static class TestParallelIndexSinglePhaseSupervisorTask extends ParallelIndexSinglePhaseSupervisorTask + private static class TestSinglePhaseParallelIndexSupervisorTask extends SinglePhaseParallelIndexSupervisorTask { - - TestParallelIndexSinglePhaseSupervisorTask( + TestSinglePhaseParallelIndexSupervisorTask( String id, TaskResource taskResource, - ParallelIndexSinglePhaseIngestionSpec ingestionSchema, + SinglePhaseParallelIndexIngestionSpec ingestionSchema, Map context, IndexingServiceClient indexingServiceClient ) { - super(id, taskResource, ingestionSchema, context, indexingServiceClient); + super( + id, + taskResource, + ingestionSchema, + context, + indexingServiceClient, + new NoopChatHandlerProvider(), + new AuthorizerMapper(ImmutableMap.of()) + ); } @Override - Iterator subTaskSpecIterator() throws IOException + Iterator subTaskSpecIterator() throws IOException { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() @@ -313,24 +335,41 @@ Iterator subTaskSpecIterator() throws IOExc return newTaskSpec((InputSplit) split); }); } + + @Override + SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + { + final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() + .getIOConfig() + .getFirehoseFactory(); + return new TestSinglePhaseParallelIndexSubTaskSpec( + getId() + "_" + getAndIncreaseNextSpecId(), + getGroupId(), + this, + new SinglePhaseParallelIndexIngestionSpec( + getIngestionSchema().getDataSchema(), + new SinglePhaseParallelIndexIOConfig( + baseFirehoseFactory.withSplit(split), + getIngestionSchema().getIOConfig().isAppendToExisting() + ), + getIngestionSchema().getTuningConfig() + ), + getContext() + ); + } } - private class LocalIndexingServiceClient extends IndexingServiceClient + private class LocalIndexingServiceClient extends NoopIndexingServiceClient { private final ConcurrentMap> tasks = new ConcurrentHashMap<>(); private final ListeningExecutorService service = MoreExecutors.listeningDecorator( Execs.multiThreaded(5, "parallel-index-single-phase-supervisor-task-test-%d") ); - LocalIndexingServiceClient() - { - super(null, null); - } - @Override public String runTask(Object taskObject) { - final ParallelIndexSinglePhaseSubTask subTask = (ParallelIndexSinglePhaseSubTask) taskObject; + final SinglePhaseParallelIndexSubTask subTask = (SinglePhaseParallelIndexSubTask) taskObject; tasks.put(subTask.getId(), service.submit(() -> { try { final TaskToolbox toolbox = createTaskToolbox(subTask); @@ -358,12 +397,30 @@ public TaskStatusResponse getTaskStatus(String taskId) final TaskStatus taskStatus = taskStatusFuture.get(); return new TaskStatusResponse( taskId, - new io.druid.client.indexing.TaskStatus(taskId, taskStatus.getStatusCode(), taskStatus.getReport(), -1) + new TaskStatusPlus( + taskId, + "index_single_phase_sub", + DateTimes.EPOCH, + DateTimes.EPOCH, + taskStatus.getStatusCode(), + taskStatus.getDuration(), + TaskLocation.unknown(), + null + ) ); } else { return new TaskStatusResponse( taskId, - new io.druid.client.indexing.TaskStatus(taskId, TaskState.RUNNING, null, -1) + new TaskStatusPlus( + taskId, + "index_single_phase_sub", + DateTimes.EPOCH, + DateTimes.EPOCH, + TaskState.RUNNING, + -1L, + TaskLocation.unknown(), + null + ) ); } } @@ -373,7 +430,16 @@ public TaskStatusResponse getTaskStatus(String taskId) System.err.println(Throwables.getStackTraceAsString(e)); return new TaskStatusResponse( taskId, - new io.druid.client.indexing.TaskStatus(taskId, TaskState.FAILED, null, -1) + new TaskStatusPlus( + taskId, + "index_single_phase_sub", + DateTimes.EPOCH, + DateTimes.EPOCH, + TaskState.FAILED, + -1L, + TaskLocation.unknown(), + null + ) ); } } else { @@ -438,4 +504,77 @@ public File getStorageDirectory() null ); } + + private static class TestSinglePhaseParallelIndexSubTaskSpec extends SinglePhaseParallelIndexSubTaskSpec + { + private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + + TestSinglePhaseParallelIndexSubTaskSpec( + String id, + String groupId, + SinglePhaseParallelIndexSupervisorTask supervisorTask, + SinglePhaseParallelIndexIngestionSpec ingestionSpec, + Map context + ) + { + super(id, groupId, supervisorTask.getId(), ingestionSpec, context); + this.supervisorTask = supervisorTask; + } + + @Override + public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) + { + return new SinglePhaseParallelIndexSubTask( + null, + getGroupId(), + null, + getSupervisorTaskId(), + numAttempts, + getIngestionSpec(), + getContext(), + null, + new LocalSinglePhaseParallelIndexTaskClientFactory(supervisorTask) + ); + } + } + + private static class LocalSinglePhaseParallelIndexTaskClientFactory + implements IndexTaskClientFactory + { + private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + + LocalSinglePhaseParallelIndexTaskClientFactory(SinglePhaseParallelIndexSupervisorTask supervisorTask) + { + this.supervisorTask = supervisorTask; + } + + @Override + public SinglePhaseParallelIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String callerId, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + return new LocalSinglePhaseParallelIndexTaskClient(callerId, supervisorTask); + } + } + + private static class LocalSinglePhaseParallelIndexTaskClient extends SinglePhaseParallelIndexTaskClient + { + private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + + public LocalSinglePhaseParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexSupervisorTask supervisorTask) + { + super(null, null, null, null, callerId, 0); + this.supervisorTask = supervisorTask; + } + + @Override + public void report(String supervisorTaskId, List pushedSegments) + { + supervisorTask.collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); + } + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java index 778221e8f6aa..89a10600c1ce 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java @@ -20,12 +20,15 @@ package io.druid.indexing.common.task; import com.google.common.util.concurrent.ListenableFuture; -import io.druid.client.indexing.IndexingServiceClient; -import io.druid.client.indexing.TaskStatus; +import io.druid.client.indexing.NoopIndexingServiceClient; import io.druid.client.indexing.TaskStatusResponse; +import io.druid.indexer.TaskLocation; import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.concurrent.Execs; import org.junit.After; import org.junit.Assert; @@ -77,7 +80,7 @@ public void testBasic() throws InterruptedException, ExecutionException, Timeout Assert.assertEquals("supervisorId", result.getSpec().getSupervisorTaskId()); Assert.assertEquals("specId" + i, result.getSpec().getId()); Assert.assertNotNull(result.getLastStatus()); - Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getState()); Assert.assertEquals(TaskState.SUCCESS, result.getLastState()); } } @@ -97,14 +100,14 @@ public void testRetry() throws InterruptedException, ExecutionException, Timeout Assert.assertEquals("specId" + i, result.getSpec().getId()); Assert.assertNotNull(result.getLastStatus()); - Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getState()); Assert.assertEquals(TaskState.SUCCESS, result.getLastState()); - final List attemptHistory = result.getAttemptHistory(); + final List attemptHistory = result.getAttemptHistory(); Assert.assertNotNull(attemptHistory); Assert.assertEquals(3, attemptHistory.size()); - Assert.assertEquals(TaskState.FAILED, attemptHistory.get(0).getStatusCode()); - Assert.assertEquals(TaskState.FAILED, attemptHistory.get(1).getStatusCode()); + Assert.assertEquals(TaskState.FAILED, attemptHistory.get(0).getState()); + Assert.assertEquals(TaskState.FAILED, attemptHistory.get(1).getState()); } } @@ -132,41 +135,34 @@ public TestTaskSpec( @Override public TestTask newSubTask(int numAttempts) { - return new TestTask(getId(), numAttempts, runTime, numFails++ < numMaxFails); + return new TestTask(getId(), runTime, numFails++ < numMaxFails); } } private static class TestTask extends NoopTask { - private final int numAttempts; private final boolean shouldFail; - TestTask(String id, int numAttempts, long runTime, boolean shouldFail) + TestTask(String id, long runTime, boolean shouldFail) { super(id, "testDataSource", runTime, 0, null, null, null); - this.numAttempts = numAttempts; this.shouldFail = shouldFail; } @Override - public io.druid.indexing.common.TaskStatus run(TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { if (shouldFail) { Thread.sleep(getRunTime()); - return io.druid.indexing.common.TaskStatus.failure(getId()); + return TaskStatus.failure(getId()); } else { return super.run(toolbox); } } } - private class TestIndexingServiceClient extends IndexingServiceClient + private class TestIndexingServiceClient extends NoopIndexingServiceClient { - TestIndexingServiceClient() - { - super(null, null); - } - @Override public String runTask(Object taskObject) { @@ -181,7 +177,16 @@ public TaskStatusResponse getTaskStatus(String taskId) { return new TaskStatusResponse( taskId, - new TaskStatus(taskId, tasks.get(taskId), null, -1) + new TaskStatusPlus( + taskId, + "testTask", + DateTimes.EPOCH, + DateTimes.EPOCH, + tasks.get(taskId), + -1L, + TaskLocation.unknown(), + "testDataSource" + ) ); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 9c6917d9dac7..b1ea297de649 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; import io.druid.indexer.TaskLocation; +import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -39,8 +40,8 @@ import io.druid.indexing.overlord.TaskRunnerWorkItem; import io.druid.indexing.overlord.TaskStorageQueryAdapter; import io.druid.java.util.common.DateTimes; -import io.druid.segment.TestHelper; import io.druid.java.util.common.Pair; +import io.druid.segment.TestHelper; import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; @@ -345,12 +346,27 @@ public void testGetTaskPayload() throws Exception public void testGetTaskStatus() throws Exception { expectAuthorizationTokenCheck(); + EasyMock.expect(taskStorageQueryAdapter.getTask("mytask")) + .andReturn(Optional.of(NoopTask.create("mytask", 0))); + EasyMock.expect(taskStorageQueryAdapter.getStatus("mytask")) - .andReturn(Optional.of(TaskStatus.success("mytask"))); + .andReturn(Optional.of(TaskStatus.running("mytask"))); + + EasyMock.expect(taskStorageQueryAdapter.getCreatedDateAndDataSource("mytask")) + .andReturn(Pair.of(DateTimes.of("2018-01-01"), "mydatasource")); + + EasyMock.expect(taskStorageQueryAdapter.getTask("othertask")) + .andReturn(Optional.absent()); EasyMock.expect(taskStorageQueryAdapter.getStatus("othertask")) .andReturn(Optional.absent()); + EasyMock.expect(taskStorageQueryAdapter.getCreatedDateAndDataSource("othertask")) + .andReturn(null); + + EasyMock.>expect(taskRunner.getKnownTasks()) + .andReturn(ImmutableList.of()); + EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); final Response response1 = overlordResource.getTaskStatus("mytask"); @@ -358,7 +374,22 @@ public void testGetTaskStatus() throws Exception TestHelper.makeJsonMapper().writeValueAsString(response1.getEntity()), TaskStatusResponse.class ); - Assert.assertEquals(new TaskStatusResponse("mytask", TaskStatus.success("mytask")), taskStatusResponse1); + Assert.assertEquals( + new TaskStatusResponse( + "mytask", + new TaskStatusPlus( + "mytask", + "noop", + DateTimes.of("2018-01-01"), + DateTimes.EPOCH, + TaskState.RUNNING, + -1L, + TaskLocation.unknown(), + "mydatasource" + ) + ), + taskStatusResponse1 + ); final Response response2 = overlordResource.getTaskStatus("othertask"); final TaskStatusResponse taskStatusResponse2 = TestHelper.makeJsonMapper().readValue( diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index f94cfd089fc6..cc85a1cb4ab8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -198,7 +198,7 @@ public void announce(DruidNode node) EmittingLogger.registerEmitter(serviceEmitter); } - @Test(timeout = 2000L) + @Test() public void testOverlordRun() throws Exception { // basic task master lifecycle test @@ -247,7 +247,7 @@ public void testOverlordRun() throws Exception Assert.assertEquals(taskId_0, ((TaskStatusResponse) response.getEntity()).getTask()); Assert.assertEquals( TaskStatus.running(taskId_0).getStatusCode(), - ((TaskStatusResponse) response.getEntity()).getStatus().getStatusCode() + ((TaskStatusResponse) response.getEntity()).getStatus().getState() ); // Simulate completion of task_0 @@ -295,7 +295,7 @@ private void waitForTaskStatus(String taskId, TaskState status) throws Interrupt { while (true) { Response response = overlordResource.getTaskStatus(taskId); - if (status.equals(((TaskStatusResponse) response.getEntity()).getStatus().getStatusCode())) { + if (status.equals(((TaskStatusResponse) response.getEntity()).getStatus().getState())) { break; } Thread.sleep(10); diff --git a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java index 7062854e1f61..f50c557351b7 100644 --- a/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java +++ b/integration-tests/src/main/java/io/druid/testing/clients/OverlordResourceTestClient.java @@ -25,6 +25,7 @@ import com.google.common.base.Predicates; import com.google.common.base.Throwables; import com.google.inject.Inject; +import io.druid.client.indexing.TaskStatusResponse; import io.druid.indexer.TaskState; import io.druid.java.util.common.ISE; import io.druid.java.util.common.RetryUtils; @@ -125,12 +126,13 @@ public TaskState getTaskStatus(String taskID) ); LOG.info("Index status response" + response.getContent()); - Map responseData = jsonMapper.readValue( - response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + TaskStatusResponse taskStatusResponse = jsonMapper.readValue( + response.getContent(), + new TypeReference() + { + } ); - //TODO: figure out a better way to parse the response... - String status = (String) ((Map) responseData.get("status")).get("status"); - return TaskState.valueOf(status); + return taskStatusResponse.getStatus().getState(); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexSinglePhaseTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITSinglePhaseParallelIndexTest.java similarity index 96% rename from integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexSinglePhaseTest.java rename to integration-tests/src/test/java/io/druid/tests/indexer/ITSinglePhaseParallelIndexTest.java index 0aea7a73e4c1..12a231dd88a4 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexSinglePhaseTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITSinglePhaseParallelIndexTest.java @@ -23,7 +23,7 @@ import org.testng.annotations.Test; @Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITParallelIndexSinglePhaseTest extends AbstractITBatchIndexTest +public class ITSinglePhaseParallelIndexTest extends AbstractITBatchIndexTest { private static String INDEX_TASK = "/indexer/wikipedia_parallel_single_phase_index_task.json"; private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_single_phase_index_queries.json"; diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json index 7495a61ca2e5..48a8df2c15c3 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json @@ -1,5 +1,5 @@ { - "type": "parallelIndexSinglePhase", + "type": "index_single_phase_parallel", "spec": { "dataSchema": { "dataSource": "wikipedia_parallel_single_phase_index_test", @@ -55,7 +55,7 @@ } }, "ioConfig": { - "type": "parallelIndexSinglePhase", + "type": "index_single_phase_parallel", "firehose": { "type": "local", "baseDir": "/resources/data/batch_index", diff --git a/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java new file mode 100644 index 000000000000..80ecee751b6c --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java @@ -0,0 +1,310 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client.indexing; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import io.druid.discovery.DruidLeaderClient; +import io.druid.indexer.TaskStatusPlus; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.jackson.JacksonUtils; +import io.druid.java.util.http.client.response.FullResponseHolder; +import io.druid.timeline.DataSegment; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import javax.ws.rs.core.MediaType; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class HttpIndexingServiceClient implements IndexingServiceClient +{ + private final DruidLeaderClient druidLeaderClient; + private final ObjectMapper jsonMapper; + + @Inject + public HttpIndexingServiceClient( + ObjectMapper jsonMapper, + @IndexingService DruidLeaderClient druidLeaderClient + ) + { + this.jsonMapper = jsonMapper; + this.druidLeaderClient = druidLeaderClient; + } + + @Override + public void mergeSegments(List segments) + { + final Iterator segmentsIter = segments.iterator(); + if (!segmentsIter.hasNext()) { + return; + } + + final String dataSource = segmentsIter.next().getDataSource(); + while (segmentsIter.hasNext()) { + DataSegment next = segmentsIter.next(); + if (!dataSource.equals(next.getDataSource())) { + throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource()); + } + } + + runTask(new ClientAppendQuery(dataSource, segments)); + } + + @Override + public void killSegments(String dataSource, Interval interval) + { + runTask(new ClientKillQuery(dataSource, interval)); + } + + @Override + public void upgradeSegment(DataSegment dataSegment) + { + runTask(new ClientConversionQuery(dataSegment)); + } + + @Override + public String compactSegments( + List segments, + int compactionTaskPriority, + @Nullable ClientCompactQueryTuningConfig tuningConfig, + @Nullable Map context + ) + { + Preconditions.checkArgument(segments.size() > 1, "Expect two or more segments to compact"); + + final String dataSource = segments.get(0).getDataSource(); + Preconditions.checkArgument( + segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)), + "Segments must have the same dataSource" + ); + + context = context == null ? new HashMap<>() : context; + context.put("priority", compactionTaskPriority); + + return runTask(new ClientCompactQuery(dataSource, segments, tuningConfig, context)); + } + + @Override + public String runTask(Object taskObject) + { + try { + final FullResponseHolder response = druidLeaderClient.go( + druidLeaderClient.makeRequest( + HttpMethod.POST, + "/druid/indexer/v1/task" + ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject)) + ); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Failed to post task[%s]", taskObject); + } + + final Map resultMap = jsonMapper.readValue( + response.getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + final String taskId = (String) resultMap.get("task"); + return Preconditions.checkNotNull(taskId, "Null task id for task[%s]", taskObject); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Override + public String killTask(String taskId) + { + try { + final FullResponseHolder response = druidLeaderClient.go( + druidLeaderClient.makeRequest( + HttpMethod.POST, + StringUtils.format("/druid/indexer/v1/task/%s/shutdown", taskId) + ) + ); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Failed to kill task[%s]", taskId); + } + + final Map resultMap = jsonMapper.readValue( + response.getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + final String killedTaskId = (String) resultMap.get("task"); + Preconditions.checkNotNull(killedTaskId, "Null task id returned for task[%s]", taskId); + Preconditions.checkState( + taskId.equals(killedTaskId), + "Requested to kill task[%s], but another task[%s] was killed!", + taskId, + killedTaskId + ); + return killedTaskId; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Override + public int getTotalWorkerCapacity() + { + try { + final FullResponseHolder response = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/workers") + .setHeader("Content-Type", MediaType.APPLICATION_JSON) + ); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while getting available cluster capacity. status[%s] content[%s]", + response.getStatus(), + response.getContent() + ); + } + final Collection workers = jsonMapper.readValue( + response.getContent(), + new TypeReference>() {} + ); + + return workers.stream().mapToInt(workerInfo -> workerInfo.getWorker().getCapacity()).sum(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public List getRunningTasks() + { + return getTasks("runningTasks"); + } + + @Override + public List getPendingTasks() + { + return getTasks("pendingTasks"); + } + + @Override + public List getWaitingTasks() + { + return getTasks("waitingTasks"); + } + + private List getTasks(String endpointSuffix) + { + try { + final FullResponseHolder responseHolder = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/%s", endpointSuffix)) + ); + + if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Error while fetching the status of the last complete task"); + } + + return jsonMapper.readValue( + responseHolder.getContent(), + new TypeReference>() + { + } + ); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + @Nullable + public TaskStatusResponse getTaskStatus(String taskId) + { + try { + final FullResponseHolder responseHolder = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/task/%s/status", taskId)) + ); + + if (responseHolder.getStatus().equals(HttpResponseStatus.OK)) { + return jsonMapper.readValue( + responseHolder.getContent(), + new TypeReference() + { + } + ); + } else { + return null; + } + } + catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + @Nullable + public TaskStatusPlus getLastCompleteTask() + { + final List completeTaskStatuses = getTasks("completeTasks?n=1"); + return completeTaskStatuses.isEmpty() ? null : completeTaskStatuses.get(0); + } + + @Override + public int killPendingSegments(String dataSource, DateTime end) + { + final String endPoint = StringUtils.format( + "/druid/indexer/v1/pendingSegments/%s?interval=%s", + dataSource, + new Interval(DateTimes.MIN, end) + ); + try { + final FullResponseHolder responseHolder = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.DELETE, endPoint) + ); + + if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Error while killing pendingSegments of dataSource[%s] created until [%s]", dataSource, end); + } + + final Map resultMap = jsonMapper.readValue( + responseHolder.getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + final Object numDeletedObject = resultMap.get("numDeleted"); + return (Integer) Preconditions.checkNotNull(numDeletedObject, "numDeletedObject"); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index 9b7c98bf8915..c03997be82a3 100644 --- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -19,279 +19,47 @@ package io.druid.client.indexing; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.inject.Inject; -import io.druid.discovery.DruidLeaderClient; import io.druid.indexer.TaskStatusPlus; -import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.IAE; -import io.druid.java.util.common.ISE; -import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.jackson.JacksonUtils; -import io.druid.java.util.http.client.response.FullResponseHolder; import io.druid.timeline.DataSegment; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; -import javax.ws.rs.core.MediaType; -import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -public class IndexingServiceClient +public interface IndexingServiceClient { - private final DruidLeaderClient druidLeaderClient; - private final ObjectMapper jsonMapper; + void killSegments(String dataSource, Interval interval); - @Inject - public IndexingServiceClient( - ObjectMapper jsonMapper, - @IndexingService DruidLeaderClient druidLeaderClient - ) - { - this.jsonMapper = jsonMapper; - this.druidLeaderClient = druidLeaderClient; - } + int killPendingSegments(String dataSource, DateTime end); - public void mergeSegments(List segments) - { - final Iterator segmentsIter = segments.iterator(); - if (!segmentsIter.hasNext()) { - return; - } + void upgradeSegment(DataSegment dataSegment); - final String dataSource = segmentsIter.next().getDataSource(); - while (segmentsIter.hasNext()) { - DataSegment next = segmentsIter.next(); - if (!dataSource.equals(next.getDataSource())) { - throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource()); - } - } + void mergeSegments(List segments); - runTask(new ClientAppendQuery(dataSource, segments)); - } - - public void killSegments(String dataSource, Interval interval) - { - runTask(new ClientKillQuery(dataSource, interval)); - } - - public void upgradeSegment(DataSegment dataSegment) - { - runTask(new ClientConversionQuery(dataSegment)); - } - - public String compactSegments( + String compactSegments( List segments, int compactionTaskPriority, @Nullable ClientCompactQueryTuningConfig tuningConfig, @Nullable Map context - ) - { - Preconditions.checkArgument(segments.size() > 1, "Expect two or more segments to compact"); - - final String dataSource = segments.get(0).getDataSource(); - Preconditions.checkArgument( - segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)), - "Segments must have the same dataSource" - ); - - context = context == null ? new HashMap<>() : context; - context.put("priority", compactionTaskPriority); - - return runTask(new ClientCompactQuery(dataSource, segments, tuningConfig, context)); - } - - public String runTask(Object taskObject) - { - try { - final FullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest( - HttpMethod.POST, - "/druid/indexer/v1/task" - ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject)) - ); - - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Failed to post task[%s]", taskObject); - } - - final Map resultMap = jsonMapper.readValue( - response.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - final String taskId = (String) resultMap.get("task"); - return Preconditions.checkNotNull(taskId, "Null task id for task[%s]", taskObject); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - public String killTask(String taskId) - { - try { - final FullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest( - HttpMethod.POST, - StringUtils.format("/druid/indexer/v1/task/%s/shutdown", taskId) - ) - ); + ); - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Failed to kill task[%s]", taskId); - } + int getTotalWorkerCapacity(); - final Map resultMap = jsonMapper.readValue( - response.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - final String killedTaskId = (String) resultMap.get("task"); - Preconditions.checkNotNull(killedTaskId, "Null task id returned for task[%s]", taskId); - Preconditions.checkState( - taskId.equals(killedTaskId), - "Requested to kill task[%s], but another task[%s] was killed!", - taskId, - killedTaskId - ); - return killedTaskId; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } + String runTask(Object taskObject); - public int getTotalWorkerCapacity() - { - try { - final FullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/workers") - .setHeader("Content-Type", MediaType.APPLICATION_JSON) - ); + String killTask(String taskId); - if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE( - "Error while getting available cluster capacity. status[%s] content[%s]", - response.getStatus(), - response.getContent() - ); - } - final Collection workers = jsonMapper.readValue( - response.getContent(), - new TypeReference>() {} - ); + List getRunningTasks(); - return workers.stream().mapToInt(workerInfo -> workerInfo.getWorker().getCapacity()).sum(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } + List getPendingTasks(); - public List getRunningTasks() - { - return getTasks("runningTasks"); - } - - public List getPendingTasks() - { - return getTasks("pendingTasks"); - } - - public List getWaitingTasks() - { - return getTasks("waitingTasks"); - } - - private List getTasks(String endpointSuffix) - { - try { - final FullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/%s", endpointSuffix)) - ); - - if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Error while fetching the status of the last complete task"); - } - - return jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference>() - { - } - ); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } + List getWaitingTasks(); @Nullable - public TaskStatusResponse getTaskStatus(String taskId) - { - try { - final FullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/task/%s/status", taskId)) - ); - - if (responseHolder.getStatus().equals(HttpResponseStatus.OK)) { - return jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference() - { - } - ); - } else { - return null; - } - } - catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - } + TaskStatusResponse getTaskStatus(String taskId); @Nullable - public TaskStatusPlus getLastCompleteTask() - { - final List completeTaskStatuses = getTasks("completeTasks?n=1"); - return completeTaskStatuses.isEmpty() ? null : completeTaskStatuses.get(0); - } - - public int killPendingSegments(String dataSource, DateTime end) - { - final String endPoint = StringUtils.format( - "/druid/indexer/v1/pendingSegments/%s?interval=%s", - dataSource, - new Interval(DateTimes.MIN, end) - ); - try { - final FullResponseHolder responseHolder = druidLeaderClient.go( - druidLeaderClient.makeRequest(HttpMethod.DELETE, endPoint) - ); - - if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Error while killing pendingSegments of dataSource[%s] created until [%s]", dataSource, end); - } - - final Map resultMap = jsonMapper.readValue( - responseHolder.getContent(), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ); - - final Object numDeletedObject = resultMap.get("numDeleted"); - return (Integer) Preconditions.checkNotNull(numDeletedObject, "numDeletedObject"); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } + TaskStatusPlus getLastCompleteTask(); } diff --git a/server/src/main/java/io/druid/client/indexing/TaskStatus.java b/server/src/main/java/io/druid/client/indexing/TaskStatus.java index 8e255ca0d6a1..4a17629709cc 100644 --- a/server/src/main/java/io/druid/client/indexing/TaskStatus.java +++ b/server/src/main/java/io/druid/client/indexing/TaskStatus.java @@ -22,11 +22,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; -import io.druid.indexer.TaskReport; import io.druid.indexer.TaskState; -import javax.annotation.Nullable; - /** * Should be synced with io.druid.indexing.common.TaskStatus */ @@ -34,20 +31,17 @@ public class TaskStatus { private final String id; private final TaskState status; - private final TaskReport report; private final long duration; @JsonCreator public TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") TaskState status, - @JsonProperty("report") @Nullable TaskReport report, @JsonProperty("duration") long duration ) { this.id = id; this.status = status; - this.report = report; this.duration = duration; // Check class invariants. @@ -67,12 +61,6 @@ public TaskState getStatusCode() return status; } - @JsonProperty("report") - public TaskReport getReport() - { - return report; - } - @JsonProperty("duration") public long getDuration() { @@ -91,14 +79,13 @@ public boolean equals(Object o) TaskStatus that = (TaskStatus) o; return duration == that.duration && java.util.Objects.equals(id, that.id) && - status == that.status && - java.util.Objects.equals(report, that.report); + status == that.status; } @Override public int hashCode() { - return java.util.Objects.hash(id, status, report, duration); + return java.util.Objects.hash(id, status, duration); } @Override @@ -107,7 +94,6 @@ public String toString() return Objects.toStringHelper(this) .add("id", id) .add("status", status) - .add("report", report) .add("duration", duration) .toString(); } diff --git a/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java index 07fed5532a17..9d84f44fca12 100644 --- a/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java +++ b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexer.TaskStatusPlus; import java.util.Objects; @@ -30,12 +31,12 @@ public class TaskStatusResponse { private final String task; // Task ID, named "task" in the JSONification of this class. - private final TaskStatus status; + private final TaskStatusPlus status; @JsonCreator public TaskStatusResponse( @JsonProperty("task") final String task, - @JsonProperty("status") final TaskStatus status + @JsonProperty("status") final TaskStatusPlus status ) { this.task = task; @@ -49,7 +50,7 @@ public String getTask() } @JsonProperty - public TaskStatus getStatus() + public TaskStatusPlus getStatus() { return status; } diff --git a/server/src/main/java/io/druid/guice/http/HttpClientModule.java b/server/src/main/java/io/druid/guice/http/HttpClientModule.java index 9d605c7bdea8..70673a3d65ab 100644 --- a/server/src/main/java/io/druid/guice/http/HttpClientModule.java +++ b/server/src/main/java/io/druid/guice/http/HttpClientModule.java @@ -23,15 +23,15 @@ import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Module; -import io.druid.java.util.http.client.HttpClient; -import io.druid.java.util.http.client.HttpClientConfig; -import io.druid.java.util.http.client.HttpClientInit; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.annotations.EscalatedClient; import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Global; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.http.client.HttpClient; +import io.druid.java.util.http.client.HttpClientConfig; +import io.druid.java.util.http.client.HttpClientInit; import io.druid.server.security.Escalator; import java.lang.annotation.Annotation; diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java b/server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java new file mode 100644 index 000000000000..350bbf667c9f --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java @@ -0,0 +1,59 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.realtime.firehose; + +import io.druid.server.security.Access; +import io.druid.server.security.Action; +import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.ForbiddenException; +import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; +import io.druid.server.security.ResourceType; + +import javax.servlet.http.HttpServletRequest; + +public class ChatHandlers +{ + /** + * Check authorization for the given action and dataSource. + * + * @return authorization result + */ + public static Access authorizationCheck( + HttpServletRequest req, + Action action, + String dataSource, + AuthorizerMapper authorizerMapper + ) + { + ResourceAction resourceAction = new ResourceAction( + new Resource(dataSource, ResourceType.DATASOURCE), + action + ); + + Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + + return access; + } +} diff --git a/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java b/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java new file mode 100644 index 000000000000..aeb6024f513f --- /dev/null +++ b/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java @@ -0,0 +1,117 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.client.indexing; + +import io.druid.indexer.TaskStatusPlus; +import io.druid.timeline.DataSegment; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + +public class NoopIndexingServiceClient implements IndexingServiceClient +{ + @Override + public void killSegments(String dataSource, Interval interval) + { + + } + + @Override + public int killPendingSegments(String dataSource, DateTime end) + { + return 0; + } + + @Override + public void upgradeSegment(DataSegment dataSegment) + { + + } + + @Override + public void mergeSegments(List segments) + { + + } + + @Override + public String compactSegments( + List segments, + int compactionTaskPriority, + @Nullable ClientCompactQueryTuningConfig tuningConfig, + @Nullable Map context + ) + { + return null; + } + + @Override + public int getTotalWorkerCapacity() + { + return 0; + } + + @Override + public String runTask(Object taskObject) + { + return null; + } + + @Override + public String killTask(String taskId) + { + return null; + } + + @Override + public List getRunningTasks() + { + return null; + } + + @Override + public List getPendingTasks() + { + return null; + } + + @Override + public List getWaitingTasks() + { + return null; + } + + @Nullable + @Override + public TaskStatusResponse getTaskStatus(String taskId) + { + return null; + } + + @Nullable + @Override + public TaskStatusPlus getLastCompleteTask() + { + return null; + } +} diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java index ddecdb2a2b9e..6258314df4d9 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java @@ -22,10 +22,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.NoopIndexingServiceClient; import io.druid.common.config.JacksonConfigManager; import io.druid.java.util.common.Intervals; +import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.server.coordinator.helper.DruidCoordinatorSegmentMerger; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; @@ -455,7 +456,7 @@ private static List> merge(final Collection segme EasyMock.replay(configManager); final List> retVal = Lists.newArrayList(); - final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null) + final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient() { @Override public void mergeSegments(List segmentsToMerge) diff --git a/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index e352f126bc86..50cdf4466bd0 100644 --- a/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import io.druid.client.indexing.ClientCompactQueryTuningConfig; import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.NoopIndexingServiceClient; import io.druid.indexer.TaskStatusPlus; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; @@ -55,7 +56,7 @@ public class DruidCoordinatorSegmentCompactorTest { private static final String DATA_SOURCE_PREFIX = "dataSource_"; - private final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null) + private final IndexingServiceClient indexingServiceClient = new NoopIndexingServiceClient() { private int compactVersionSuffix = 0; private int idSuffix = 0; diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 33784eb7244a..aaea01bcd3b7 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -33,6 +33,7 @@ import io.druid.client.CoordinatorServerView; import io.druid.client.HttpServerInventoryViewResource; import io.druid.client.coordinator.Coordinator; +import io.druid.client.indexing.HttpIndexingServiceClient; import io.druid.client.indexing.IndexingServiceClient; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.ConditionalMultibind; @@ -167,7 +168,7 @@ public void configure(Binder binder) .toProvider(AuditManagerProvider.class) .in(ManageLifecycle.class); - binder.bind(IndexingServiceClient.class).in(LazySingleton.class); + binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); binder.bind(CoordinatorServerView.class).in(LazySingleton.class); binder.bind(LookupCoordinatorManager.class).in(LazySingleton.class); diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 58ef96bb20eb..6d123803ab07 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -24,9 +24,11 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.TypeLiteral; import com.google.inject.name.Names; import com.google.inject.util.Providers; import io.airlift.airline.Command; +import io.druid.client.indexing.IndexingServiceClient; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.WorkerNodeService; import io.druid.guice.IndexingServiceFirehoseModule; @@ -39,6 +41,8 @@ import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Self; import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.task.IndexTaskClientFactory; +import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClient; import io.druid.indexing.overlord.ForkingTaskRunner; import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.worker.Worker; @@ -91,6 +95,9 @@ public void configure(Binder binder) binder.bind(TaskRunner.class).to(ForkingTaskRunner.class); binder.bind(ForkingTaskRunner.class).in(LazySingleton.class); + binder.bind(IndexingServiceClient.class).toProvider(Providers.of(null)); + binder.bind(new TypeLiteral>(){}) + .toProvider(Providers.of(null)); binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(null)); binder.bind(WorkerTaskMonitor.class).in(ManageLifecycle.class); diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index a6b7fbfab9bd..7a4f9b289f07 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -35,6 +35,7 @@ import io.airlift.airline.Command; import io.druid.audit.AuditManager; import io.druid.client.indexing.IndexingService; +import io.druid.client.indexing.IndexingServiceClient; import io.druid.client.indexing.IndexingServiceSelectorConfig; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.IndexingServiceFirehoseModule; @@ -54,12 +55,13 @@ import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.common.task.IndexTaskClientFactory; +import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClient; import io.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import io.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; import io.druid.indexing.overlord.ForkingTaskRunnerFactory; import io.druid.indexing.overlord.HeapMemoryTaskStorage; import io.druid.indexing.overlord.IndexerMetadataStorageAdapter; -import io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerFactory; import io.druid.indexing.overlord.MetadataTaskStorage; import io.druid.indexing.overlord.RemoteTaskRunnerFactory; import io.druid.indexing.overlord.TaskLockbox; @@ -77,6 +79,7 @@ import io.druid.indexing.overlord.helpers.OverlordHelper; import io.druid.indexing.overlord.helpers.TaskLogAutoCleaner; import io.druid.indexing.overlord.helpers.TaskLogAutoCleanerConfig; +import io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerFactory; import io.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerResource; import io.druid.indexing.overlord.http.OverlordRedirectInfo; import io.druid.indexing.overlord.http.OverlordResource; @@ -181,7 +184,10 @@ public void configure(Binder binder) binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); - binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(null)); + binder.bind(IndexingServiceClient.class).toProvider(Providers.of(null)); + binder.bind(new TypeLiteral>(){}) + .toProvider(Providers.of(null)); + binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(null)); configureTaskStorage(binder); configureAutoscale(binder); diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index f3107a2626e1..cae4609c85f8 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -28,6 +28,7 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; import com.google.inject.name.Names; @@ -36,6 +37,7 @@ import io.airlift.airline.Option; import io.druid.client.cache.CacheConfig; import io.druid.client.coordinator.CoordinatorClient; +import io.druid.client.indexing.HttpIndexingServiceClient; import io.druid.client.indexing.IndexingServiceClient; import io.druid.guice.Binders; import io.druid.guice.CacheModule; @@ -62,6 +64,9 @@ import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; +import io.druid.indexing.common.task.IndexTaskClientFactory; +import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClient; +import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClientFactory; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.HeapMemoryTaskStorage; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -169,7 +174,11 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class); configureTaskActionClient(binder); - binder.bind(IndexingServiceClient.class).in(LazySingleton.class); + binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); + + binder.bind(new TypeLiteral>(){}) + .to(SinglePhaseParallelIndexTaskClientFactory.class) + .in(LazySingleton.class); binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); From a83c76acd7a7ff927944d9cb18e635161ae5f99a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 26 Mar 2018 19:30:18 -0700 Subject: [PATCH 09/47] fix build --- .../main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 1032f633240a..ea75809cf5e2 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -261,7 +261,7 @@ protected Predicate getRetryCondition() } @Override - public FiniteFirehoseFactory withSplit(InputSplit split) + public FiniteFirehoseFactory withSplit(InputSplit split) { final String authority = split.get().getBucketName(); final String path = split.get().getKey(); From 8d68dba5949b65f501107bddd2347f24063f7465 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 27 Mar 2018 14:41:29 -0700 Subject: [PATCH 10/47] add docs --- .../development/extensions-contrib/azure.md | 2 + .../extensions-contrib/cloudfiles.md | 2 + .../development/extensions-contrib/google.md | 2 + .../content/development/extensions-core/s3.md | 1 + docs/content/ingestion/firehose.md | 2 + docs/content/ingestion/native_tasks.md | 297 ++++++++++++++++++ docs/content/ingestion/tasks.md | 224 ++++--------- .../actions/SegmentInsertActionTest.java | 2 +- .../SegmentTransactionalInsertActionTest.java | 2 +- 9 files changed, 362 insertions(+), 172 deletions(-) create mode 100644 docs/content/ingestion/native_tasks.md diff --git a/docs/content/development/extensions-contrib/azure.md b/docs/content/development/extensions-contrib/azure.md index 3a3946accf48..48b739941e54 100644 --- a/docs/content/development/extensions-contrib/azure.md +++ b/docs/content/development/extensions-contrib/azure.md @@ -33,6 +33,8 @@ The storage account is shared with the one used for Azure deep storage functiona As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). + Sample spec: ```json diff --git a/docs/content/development/extensions-contrib/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md index 954179d5fbdb..f5691b7eac4d 100644 --- a/docs/content/development/extensions-contrib/cloudfiles.md +++ b/docs/content/development/extensions-contrib/cloudfiles.md @@ -32,6 +32,8 @@ The storage account is shared with the one used for Racksapce's Cloud Files deep As with the Azure blobstore, it is assumed to be gzipped if the extension ends in .gz +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). + Sample spec: ```json diff --git a/docs/content/development/extensions-contrib/google.md b/docs/content/development/extensions-contrib/google.md index 7aa764728828..2ea55c02293f 100644 --- a/docs/content/development/extensions-contrib/google.md +++ b/docs/content/development/extensions-contrib/google.md @@ -23,6 +23,8 @@ This firehose ingests events, similar to the StaticS3Firehose, but from an Googl As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). + Sample spec: ```json diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index 3d32c9de8953..dbf2b553c59b 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -28,6 +28,7 @@ S3-compatible deep storage is basically either S3 or something like Google Stora ## StaticS3Firehose This firehose ingests events from a predefined list of S3 objects. +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). Sample spec: diff --git a/docs/content/ingestion/firehose.md b/docs/content/ingestion/firehose.md index e1c2e73931e2..d974b74ab167 100644 --- a/docs/content/ingestion/firehose.md +++ b/docs/content/ingestion/firehose.md @@ -20,6 +20,7 @@ For additional firehoses, please see our [extensions list](../development/extens This Firehose can be used to read the data from files on local disk. It can be used for POCs to ingest data on disk. +This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#single-phase-parallel-index-task). A sample local firehose spec is shown below: ```json @@ -39,6 +40,7 @@ A sample local firehose spec is shown below: #### HttpFirehose This Firehose can be used to read the data from remote sites via HTTP. +This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#single-phase-parallel-index-task). A sample http firehose spec is shown below: ```json diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md new file mode 100644 index 000000000000..a06bf4025456 --- /dev/null +++ b/docs/content/ingestion/native_tasks.md @@ -0,0 +1,297 @@ +--- +layout: doc_page +--- +# Native Index Tasks + +Druid currently two types of native index tasks, i.e., `local index task` and `single phase parallel index task`. + +Single Phase Parallel Index Task +-------------------------------- + +The Single Phase Parallel Index Task is a task for parallel batch indexing. This task only uses Druid's resource and +doesn't depend on other external systems like Hadoop. This task works in a single phase without shuffling intermediate +data. `index_single_phase_parallel` task is a supervisor task which basically creates multiple worker tasks and submits +them to overlords. Each worker task reads input data and makes segments. Once they successfully generate segments for all +input, they report the generated segment list to the supervisor task. The supervisor task periodically checks the worker +task statuses. If one of them fails, it retries the failed task until the retrying number reaches the configured limit. +If all worker tasks succeed, then it collects the reported list of generated segments and publishes those segments at once. + +To use this task, the `firehose` in `ioConfig` should be _splittable_. If it's not, this task runs sequentially. The +current splittable fireshoses are [`LocalFirehose`](./firehose.html#localfirehose), [`HttpFirehose`](./firehose.html#httpfirehose) +, [`StaticS3Firehose`](../development/extensions-core/statics3firehose), [`StaticAzureBlobStoreFirehose`](../development/extensions-contrib/staticazureblobstorefirehose) +, [`StaticGoogleBlobStoreFirehose`](../development/extensions-contrib/staticgoogleblobstorefirehose), and [`StaticCloudFilesFirehose`](../development/extensions-contrib/staticcloudfilesfirehose). + +The splittable firehose is responsible for generating _splits_ which is an input of each worker tasks generated by the +supervisor task. As a result, the number of worker tasks depends on the implementation of splittable firehoses. + +Since this task doesn't shuffle intermediate data, it isn't available for [perfect rollup](../design/index.html). + +An example ingestion spec is: + +```json +{ + "type": "index_single_phase_parallel", + "spec": { + "dataSchema": { + "dataSource": "wikipedia_parallel_single_phase_index_test", + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": "second", + "intervals" : [ "2013-08-31/2013-09-02" ] + }, + "parser": { + "parseSpec": { + "format" : "json", + "timestampSpec": { + "column": "timestamp" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ] + } + } + } + }, + "ioConfig": { + "type": "index_single_phase_parallel", + "firehose": { + "type": "local", + "baseDir": "examples/indexing/", + "filter": "wikipedia_index_data*" + } + } + } +} +``` + +#### Task Properties + +|property|description|required?| +|--------|-----------|---------| +|type|The task type, this should always be `index_single_phase_parallel`.|yes| +|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no| +|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes| +|context|Context containing various task configuration parameters. See below for more details.|no| + +#### DataSchema + +This field is required. + +See [Ingestion](../ingestion/index.html) + +#### IOConfig + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|The task type, this should always be `index_single_phase_parallel`.|none|yes| +|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes| +|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no| + +#### TuningConfig + +The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details. + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|The task type, this should always be `index_single_phase_parallel`.|none|yes| +|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| +|maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|75000|no| +|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| +|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| +|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| +|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| +|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| +|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| +|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no| +|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no| +|maxNumBatchTasks|Maximum number of tasks which can be run at the same time.|Integer.MAX_VALUE|no| +|maxRetry|Maximum number of retries on task failures.|3|no| +|taskStatusCheckPeriodMs|Polling period in milleseconds to check running task statuses.|1000|no| +|chatHandlerTimeout|Timeout for reporting the pushed segments in worker tasks.|PT10S|no| +|chatHandlerNumRetries|Retries for reporting the pushed segments in worker tasks.|5|no| + +Local Index Task +---------------- + +The Local Index Task is designed to be used for smaller data sets. The task executes within the indexing service. The grammar of the index task is as follows: + +```json +{ + "type" : "index", + "spec" : { + "dataSchema" : { + "dataSource" : "wikipedia", + "parser" : { + "type" : "string", + "parseSpec" : { + "format" : "json", + "timestampSpec" : { + "column" : "timestamp", + "format" : "auto" + }, + "dimensionsSpec" : { + "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], + "dimensionExclusions" : [], + "spatialDimensions" : [] + } + } + }, + "metricsSpec" : [ + { + "type" : "count", + "name" : "count" + }, + { + "type" : "doubleSum", + "name" : "added", + "fieldName" : "added" + }, + { + "type" : "doubleSum", + "name" : "deleted", + "fieldName" : "deleted" + }, + { + "type" : "doubleSum", + "name" : "delta", + "fieldName" : "delta" + } + ], + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "DAY", + "queryGranularity" : "NONE", + "intervals" : [ "2013-08-31/2013-09-01" ] + } + }, + "ioConfig" : { + "type" : "index", + "firehose" : { + "type" : "local", + "baseDir" : "examples/indexing/", + "filter" : "wikipedia_data.json" + } + }, + "tuningConfig" : { + "type" : "index", + "targetPartitionSize" : 5000000, + "maxRowsInMemory" : 75000 + } + } +} +``` + +#### Task Properties + +|property|description|required?| +|--------|-----------|---------| +|type|The task type, this should always be "index".|yes| +|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no| +|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes| +|context|Context containing various task configuration parameters. See below for more details.|no| + +#### DataSchema + +This field is required. + +See [Ingestion](../ingestion/index.html) + +#### IOConfig + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|The task type, this should always be "index".|none|yes| +|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes| +|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no| + +#### TuningConfig + +The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details. + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|The task type, this should always be "index".|none|yes| +|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| +|maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|75000|no| +|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| +|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| +|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| +|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| +|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| +|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| +|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| +|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no| +|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no| + +Common Configurations +--------------------- + +#### IndexSpec + +The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column +compression formats. The indexSpec is optional and default parameters will be used if not specified. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| +|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| +|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| + +##### Bitmap types + +For Concise bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|type|String|Must be `concise`.|yes| + +For Roaring bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|type|String|Must be `roaring`.|yes| +|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)| + +#### Segment publishing modes + +While ingesting data using the Index task, it creates segments from the input data and publishes them. For segment publishing, the Index task supports two segment publishing modes, i.e., _bulk publishing mode_ and _incremental publishing mode_ for [perfect rollup and best-effort rollup](./design/index.html), respectively. + +In the bulk publishing mode, every segment is published at the very end of the index task. Until then, created segments are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a problem due to limited storage capacity, and is not recommended to use in production. + +On the contrary, in the incremental publishing mode, segments are incrementally published, that is they can be published in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds, the index task immediately publishes all segments created until that moment, cleans all published segments up, and continues to ingest remaining data. + +To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig. diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 4d424efcadb4..fc39c9ef4b7c 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -13,177 +13,10 @@ Segment Creation Tasks See [batch ingestion](../ingestion/batch-ingestion.html). -### Index Task +### Native Index Tasks -The Index Task is a simpler variation of the Index Hadoop task that is designed to be used for smaller data sets. The task executes within the indexing service and does not require an external Hadoop setup to use. The grammar of the index task is as follows: - -```json -{ - "type" : "index", - "spec" : { - "dataSchema" : { - "dataSource" : "wikipedia", - "parser" : { - "type" : "string", - "parseSpec" : { - "format" : "json", - "timestampSpec" : { - "column" : "timestamp", - "format" : "auto" - }, - "dimensionsSpec" : { - "dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"], - "dimensionExclusions" : [], - "spatialDimensions" : [] - } - } - }, - "metricsSpec" : [ - { - "type" : "count", - "name" : "count" - }, - { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, - { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, - { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" - } - ], - "granularitySpec" : { - "type" : "uniform", - "segmentGranularity" : "DAY", - "queryGranularity" : "NONE", - "intervals" : [ "2013-08-31/2013-09-01" ] - } - }, - "ioConfig" : { - "type" : "index", - "firehose" : { - "type" : "local", - "baseDir" : "examples/indexing/", - "filter" : "wikipedia_data.json" - } - }, - "tuningConfig" : { - "type" : "index", - "targetPartitionSize" : 5000000, - "maxRowsInMemory" : 75000 - } - } -} -``` - -#### Task Properties - -|property|description|required?| -|--------|-----------|---------| -|type|The task type, this should always be "index".|yes| -|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no| -|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes| -|context|Context containing various task configuration parameters. See below for more details.|no| - -#### Task Priority - -Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking) - -Each task has a priority which is used for lock acquisition. The locks of higher-priority tasks can preempt the locks of lower-priority tasks if they try to acquire for the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted. - -Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority. - -|task type|default priority| -|---------|----------------| -|Realtime index task|75| -|Batch index task|50| -|Merge/Append/Compaction task|25| -|Other tasks|0| - -You can override the task priority by setting your priority in the task context like below. - -```json -"context" : { - "priority" : 100 -} -``` - -#### DataSchema - -This field is required. - -See [Ingestion](../ingestion/index.html) - -#### IOConfig - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|The task type, this should always be "index".|none|yes| -|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes| -|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no| - -#### TuningConfig - -The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details. - -|property|description|default|required?| -|--------|-----------|-------|---------| -|type|The task type, this should always be "index".|none|yes| -|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| -|maxRowsInMemory|Used in determining when intermediate persists to disk should occur.|75000|no| -|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| -|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| -|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| -|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| -|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| -|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| -|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| -|publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no| -|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no| - -#### IndexSpec - -The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column -compression formats. The indexSpec is optional and default parameters will be used if not specified. - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| -|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| -|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| -|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| - -##### Bitmap types - -For Concise bitmaps: - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|type|String|Must be `concise`.|yes| - -For Roaring bitmaps: - -|Field|Type|Description|Required| -|-----|----|-----------|--------| -|type|String|Must be `roaring`.|yes| -|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)| - -#### Segment publishing modes - -While ingesting data using the Index task, it creates segments from the input data and publishes them. For segment publishing, the Index task supports two segment publishing modes, i.e., _bulk publishing mode_ and _incremental publishing mode_ for [perfect rollup and best-effort rollup](./design/index.html), respectively. - -In the bulk publishing mode, every segment is published at the very end of the index task. Until then, created segments are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a problem due to limited storage capacity, and is not recommended to use in production. - -On the contrary, in the incremental publishing mode, segments are incrementally published, that is they can be published in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds, the index task immediately publishes all segments created until that moment, cleans all published segments up, and continues to ingest remaining data. - -To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig. +Druid provides a native index tasks which doesn't need any dependencies on other systems. +See [native index tasks](./native_tasks.html) for more details. Segment Merging Tasks --------------------- @@ -378,6 +211,33 @@ The values are described below. Unlike the hadoop convert task, the indexing service task draws its output path from the indexing service's configuration. +#### IndexSpec + +The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column +compression formats. The indexSpec is optional and default parameters will be used if not specified. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| +|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| +|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| + +##### Bitmap types + +For Concise bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|type|String|Must be `concise`.|yes| + +For Roaring bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|type|String|Must be `roaring`.|yes| +|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)| + ### Noop Task These tasks start, sleep for a time and are used only for testing. The available grammar is: @@ -424,3 +284,27 @@ A task writing data into a dataSource must acquire exclusive locks for target in Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Task ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a "task group", which is a set of tasks that can share interval locks. + +Task Priority +------------- + +Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking) + +Each task has a priority which is used for lock acquisition. The locks of higher-priority tasks can preempt the locks of lower-priority tasks if they try to acquire for the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted. + +Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority. + +|task type|default priority| +|---------|----------------| +|Realtime index task|75| +|Batch index task|50| +|Merge/Append/Compaction task|25| +|Other tasks|0| + +You can override the task priority by setting your priority in the task context like below. + +```json +"context" : { + "priority" : 100 +} +``` diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java index fb6df29e602a..338056a8981a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -127,7 +127,7 @@ public void testFailBadVersion() throws Exception actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); thrown.expect(IllegalStateException.class); - thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task")); + thrown.expectMessage(CoreMatchers.containsString("are not covered by locks")); final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection( task, Collections.singletonList(INTERVAL), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java index 859752ecbd74..948be817eae1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java @@ -155,7 +155,7 @@ public void testFailBadVersion() throws Exception actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); thrown.expect(IllegalStateException.class); - thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task")); + thrown.expectMessage(CoreMatchers.containsString("are not covered by locks")); SegmentPublishResult result = action.perform(task, actionTestKit.getTaskActionToolbox()); Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(SEGMENT3), true), result); } From 00be56c2b7a2e0aa82f3d67f57a09123b0595e33 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 28 Mar 2018 15:35:58 -0700 Subject: [PATCH 11/47] fix ITUnionQueryTest --- .../java/io/druid/tests/indexer/ITUnionQueryTest.java | 6 ++++-- .../data/union_query/wikipedia_index_data.json | 10 ++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) create mode 100644 integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java index f0209b71778e..c0bbf4b4df0f 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITUnionQueryTest.java @@ -47,7 +47,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest private static final Logger LOG = new Logger(ITUnionQueryTest.class); private static final String UNION_TASK_RESOURCE = "/indexer/wikipedia_union_index_task.json"; private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName"; - private static final String UNION_DATA_FILE = "/data/batch_index/wikipedia_index_data1.json"; + private static final String UNION_DATA_FILE = "/data/union_query/wikipedia_index_data.json"; private static final String UNION_QUERIES_RESOURCE = "/indexer/union_queries.json"; private static final String UNION_DATASOURCE = "wikipedia_index_test"; @@ -91,7 +91,9 @@ public void testUnionQuery() RetryUtil.retryUntil( () -> { for (int i = 0; i < numTasks; i++) { - if (queryHelper.countRows(UNION_DATASOURCE + i, "2013-08-31/2013-09-01") < 5) { + final int countRows = queryHelper.countRows(UNION_DATASOURCE + i, "2013-08-31/2013-09-01"); + if (countRows < 5) { + LOG.warn("%d events have been ingested to %s so far", countRows, UNION_DATASOURCE + i); return false; } } diff --git a/integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json b/integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json new file mode 100644 index 000000000000..b186657dbcf0 --- /dev/null +++ b/integration-tests/src/test/resources/data/union_query/wikipedia_index_data.json @@ -0,0 +1,10 @@ +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143} +{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} \ No newline at end of file From 3a94432bb0c15b080be839bf1156d42f0e4a0c1c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 9 Apr 2018 10:30:50 -0700 Subject: [PATCH 12/47] fix failures --- .../druid/indexing/common/task/IndexTask.java | 10 ++- .../task/SinglePhaseParallelIndexSubTask.java | 70 ++++++++++++++----- ...ePhaseParallelIndexSupervisorTaskTest.java | 2 +- 3 files changed, 59 insertions(+), 23 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 12a3bdd1758e..80b049311e3e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -777,7 +777,7 @@ private Map> collectIntervalsAndShardSp determinePartitionsFireDepartmentMetrics.incrementUnparseable(); if (determinePartitionsFireDepartmentMetrics.unparseable() > ingestionSchema.getTuningConfig() - .getMaxParseExceptions()) { + .getMaxParseExceptions()) { throw new RuntimeException("Max parse exceptions exceeded, terminating task..."); } } @@ -1362,12 +1362,16 @@ private IndexTuningConfig( this.maxParseExceptions = 0; this.maxSavedParseExceptions = maxSavedParseExceptions == null ? 0 : Math.min(1, maxSavedParseExceptions); } else { - this.maxParseExceptions = maxParseExceptions == null ? TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : maxParseExceptions; + this.maxParseExceptions = maxParseExceptions == null ? + TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS : + maxParseExceptions; this.maxSavedParseExceptions = maxSavedParseExceptions == null ? TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS : maxSavedParseExceptions; } - this.logParseExceptions = logParseExceptions == null ? TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : logParseExceptions; + this.logParseExceptions = logParseExceptions == null ? + TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS : + logParseExceptions; } private static Integer initializeTargetPartitionSize(Integer numShards, Integer targetPartitionSize) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java index 62783dc5a7c7..98eee6002dbb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java @@ -105,9 +105,13 @@ public class SinglePhaseParallelIndexSubTask extends AbstractTask private final String supervisorTaskId; private final IndexingServiceClient indexingServiceClient; private final IndexTaskClientFactory taskClientFactory; + @Nullable private final CircularBuffer determinePartitionsSavedParseExceptions; + @Nullable private final CircularBuffer buildSegmentsSavedParseExceptions; + private final FireDepartment fireDepartmentForMetrics; + private IngestionState ingestionState; @JsonCreator @@ -142,11 +146,22 @@ public SinglePhaseParallelIndexSubTask( this.indexingServiceClient = indexingServiceClient; this.taskClientFactory = taskClientFactory; - determinePartitionsSavedParseExceptions = new CircularBuffer<>( - ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() - ); - buildSegmentsSavedParseExceptions = new CircularBuffer<>( - ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() + if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { + determinePartitionsSavedParseExceptions = new CircularBuffer<>( + ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() + ); + buildSegmentsSavedParseExceptions = new CircularBuffer<>( + ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() + ); + } else { + determinePartitionsSavedParseExceptions = null; + buildSegmentsSavedParseExceptions = null; + } + + this.fireDepartmentForMetrics = new FireDepartment( + ingestionSchema.getDataSchema(), + new RealtimeIOConfig(null, null, null), + null ); this.ingestionState = IngestionState.NOT_STARTED; @@ -224,7 +239,11 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception FileUtils.forceMkdir(firehoseTempDir); ingestionState = IngestionState.DETERMINE_PARTITIONS; - final IndexTask.ShardSpecs shardSpecs = determineShardSpecs(firehoseFactory, firehoseTempDir); + final IndexTask.ShardSpecs shardSpecs = determineShardSpecs( + firehoseFactory, + firehoseTempDir, + fireDepartmentForMetrics.getMetrics() + ); final DataSchema dataSchema; final Map versions; @@ -307,7 +326,8 @@ private Map tryAcquireExclusiveSurrogateLocks( */ private IndexTask.ShardSpecs determineShardSpecs( final FirehoseFactory firehoseFactory, - final File firehoseTempDir + final File firehoseTempDir, + FireDepartmentMetrics fireDepartmentMetrics ) throws IOException { final IndexTask.IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); @@ -331,7 +351,9 @@ private IndexTask.ShardSpecs determineShardSpecs( firehoseFactory, firehoseTempDir, granularitySpec, - determineIntervals + determineIntervals, + determinePartitionsSavedParseExceptions, + fireDepartmentMetrics ); } } @@ -353,7 +375,9 @@ private static IndexTask.ShardSpecs createShardSpecsFromInput( FirehoseFactory firehoseFactory, File firehoseTempDir, GranularitySpec granularitySpec, - boolean determineIntervals + boolean determineIntervals, + CircularBuffer determinePartitionsSavedParseExceptions, + FireDepartmentMetrics fireDepartmentMetrics ) throws IOException { log.info("Determining intervals"); @@ -364,7 +388,9 @@ private static IndexTask.ShardSpecs createShardSpecsFromInput( firehoseFactory, firehoseTempDir, granularitySpec, - determineIntervals + determineIntervals, + determinePartitionsSavedParseExceptions, + fireDepartmentMetrics ); final Map> intervalToShardSpecs = intervals @@ -380,7 +406,9 @@ private static List collectIntervalsAndShardSpecs( FirehoseFactory firehoseFactory, File firehoseTempDir, GranularitySpec granularitySpec, - boolean determineIntervals + boolean determineIntervals, + CircularBuffer determinePartitionsSavedParseExceptions, + FireDepartmentMetrics fireDepartmentMetrics ) throws IOException { final List intervals = new ArrayList<>(); @@ -416,10 +444,17 @@ private static List collectIntervalsAndShardSpecs( } } catch (ParseException e) { - if (ingestionSchema.getTuningConfig().isReportParseExceptions()) { - throw e; - } else { - unparseable++; + if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { + log.error(e, "Encountered parse exception: "); + } + + if (determinePartitionsSavedParseExceptions != null) { + determinePartitionsSavedParseExceptions.add(e); + } + + fireDepartmentMetrics.incrementUnparseable(); + if (fireDepartmentMetrics.unparseable() > ingestionSchema.getTuningConfig().getMaxParseExceptions()) { + throw new RuntimeException("Max parse exceptions exceeded, terminating task...", e); } } } @@ -465,10 +500,6 @@ private List generateAndPushSegments( ) throws IOException, InterruptedException { final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); - final FireDepartment fireDepartmentForMetrics = new FireDepartment( - dataSchema, new RealtimeIOConfig(null, null, null), null - ); - final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); if (toolbox.getMonitorScheduler() != null) { toolbox.getMonitorScheduler().addMonitor( @@ -495,6 +526,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null versions ); } + final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); try ( final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java index 78b47e201d23..aea25e9e1ae7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java @@ -508,7 +508,7 @@ public File getStorageDirectory() null, null, null, - null + new NoopTestTaskFileWriter() ); } From c68b21a940279e41a46b5e9a1779456221914196 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 10 Apr 2018 11:40:46 -0700 Subject: [PATCH 13/47] disable metrics reporting --- .../task/SinglePhaseParallelIndexSubTask.java | 113 ++++-------------- ...inglePhaseParallelIndexSupervisorTask.java | 10 ++ 2 files changed, 30 insertions(+), 93 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java index 98eee6002dbb..83380951741b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java @@ -31,7 +31,6 @@ import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.indexer.IngestionState; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.appenderator.CountingActionBasedSegmentAllocator; @@ -71,7 +70,6 @@ import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.ShardSpec; -import io.druid.utils.CircularBuffer; import org.codehaus.plexus.util.FileUtils; import org.joda.time.Interval; @@ -105,14 +103,6 @@ public class SinglePhaseParallelIndexSubTask extends AbstractTask private final String supervisorTaskId; private final IndexingServiceClient indexingServiceClient; private final IndexTaskClientFactory taskClientFactory; - @Nullable - private final CircularBuffer determinePartitionsSavedParseExceptions; - @Nullable - private final CircularBuffer buildSegmentsSavedParseExceptions; - - private final FireDepartment fireDepartmentForMetrics; - - private IngestionState ingestionState; @JsonCreator public SinglePhaseParallelIndexSubTask( @@ -145,26 +135,6 @@ public SinglePhaseParallelIndexSubTask( this.supervisorTaskId = supervisorTaskId; this.indexingServiceClient = indexingServiceClient; this.taskClientFactory = taskClientFactory; - - if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { - determinePartitionsSavedParseExceptions = new CircularBuffer<>( - ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() - ); - buildSegmentsSavedParseExceptions = new CircularBuffer<>( - ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() - ); - } else { - determinePartitionsSavedParseExceptions = null; - buildSegmentsSavedParseExceptions = null; - } - - this.fireDepartmentForMetrics = new FireDepartment( - ingestionSchema.getDataSchema(), - new RealtimeIOConfig(null, null, null), - null - ); - - this.ingestionState = IngestionState.NOT_STARTED; } @Override @@ -238,12 +208,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Firehose temporary directory is automatically removed when this IndexTask completes. FileUtils.forceMkdir(firehoseTempDir); - ingestionState = IngestionState.DETERMINE_PARTITIONS; - final IndexTask.ShardSpecs shardSpecs = determineShardSpecs( - firehoseFactory, - firehoseTempDir, - fireDepartmentForMetrics.getMetrics() - ); + final IndexTask.ShardSpecs shardSpecs = determineShardSpecs(firehoseFactory, firehoseTempDir); final DataSchema dataSchema; final Map versions; @@ -270,7 +235,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception dataSchema = ingestionSchema.getDataSchema(); } - ingestionState = IngestionState.BUILD_SEGMENTS; final List pushedSegments = generateAndPushSegments( toolbox, dataSchema, @@ -288,7 +252,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ); taskClient.report(supervisorTaskId, pushedSegments); - ingestionState = IngestionState.COMPLETED; return TaskStatus.success(getId()); } @@ -326,8 +289,7 @@ private Map tryAcquireExclusiveSurrogateLocks( */ private IndexTask.ShardSpecs determineShardSpecs( final FirehoseFactory firehoseFactory, - final File firehoseTempDir, - FireDepartmentMetrics fireDepartmentMetrics + final File firehoseTempDir ) throws IOException { final IndexTask.IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); @@ -351,9 +313,7 @@ private IndexTask.ShardSpecs determineShardSpecs( firehoseFactory, firehoseTempDir, granularitySpec, - determineIntervals, - determinePartitionsSavedParseExceptions, - fireDepartmentMetrics + determineIntervals ); } } @@ -375,9 +335,7 @@ private static IndexTask.ShardSpecs createShardSpecsFromInput( FirehoseFactory firehoseFactory, File firehoseTempDir, GranularitySpec granularitySpec, - boolean determineIntervals, - CircularBuffer determinePartitionsSavedParseExceptions, - FireDepartmentMetrics fireDepartmentMetrics + boolean determineIntervals ) throws IOException { log.info("Determining intervals"); @@ -388,9 +346,7 @@ private static IndexTask.ShardSpecs createShardSpecsFromInput( firehoseFactory, firehoseTempDir, granularitySpec, - determineIntervals, - determinePartitionsSavedParseExceptions, - fireDepartmentMetrics + determineIntervals ); final Map> intervalToShardSpecs = intervals @@ -406,9 +362,7 @@ private static List collectIntervalsAndShardSpecs( FirehoseFactory firehoseFactory, File firehoseTempDir, GranularitySpec granularitySpec, - boolean determineIntervals, - CircularBuffer determinePartitionsSavedParseExceptions, - FireDepartmentMetrics fireDepartmentMetrics + boolean determineIntervals ) throws IOException { final List intervals = new ArrayList<>(); @@ -444,17 +398,10 @@ private static List collectIntervalsAndShardSpecs( } } catch (ParseException e) { - if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { - log.error(e, "Encountered parse exception: "); - } - - if (determinePartitionsSavedParseExceptions != null) { - determinePartitionsSavedParseExceptions.add(e); - } - - fireDepartmentMetrics.incrementUnparseable(); - if (fireDepartmentMetrics.unparseable() > ingestionSchema.getTuningConfig().getMaxParseExceptions()) { - throw new RuntimeException("Max parse exceptions exceeded, terminating task...", e); + if (ingestionSchema.getTuningConfig().isReportParseExceptions()) { + throw e; + } else { + unparseable++; } } } @@ -500,6 +447,10 @@ private List generateAndPushSegments( ) throws IOException, InterruptedException { final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + dataSchema, new RealtimeIOConfig(null, null, null), null + ); + final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); if (toolbox.getMonitorScheduler() != null) { toolbox.getMonitorScheduler().addMonitor( @@ -526,7 +477,6 @@ private List generateAndPushSegments( versions ); } - final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); try ( final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); @@ -579,14 +529,14 @@ private List generateAndPushSegments( throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); } - if (addResult.getParseException() != null) { - handleParseException(fireDepartmentMetrics, addResult.getParseException()); - } else { - fireDepartmentMetrics.incrementProcessed(); - } + fireDepartmentMetrics.incrementProcessed(); } catch (ParseException e) { - handleParseException(fireDepartmentMetrics, e); + if (tuningConfig.isReportParseExceptions()) { + throw e; + } else { + fireDepartmentMetrics.incrementUnparseable(); + } } } @@ -601,29 +551,6 @@ private List generateAndPushSegments( } } - private void handleParseException(FireDepartmentMetrics fireDepartmentMetrics, ParseException e) - { - if (e.isFromPartiallyValidRow()) { - fireDepartmentMetrics.incrementProcessedWithErrors(); - } else { - fireDepartmentMetrics.incrementUnparseable(); - } - - if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { - log.error(e, "Encountered parse exception:"); - } - - if (buildSegmentsSavedParseExceptions != null) { - buildSegmentsSavedParseExceptions.add(e); - } - - if (fireDepartmentMetrics.unparseable() - + fireDepartmentMetrics.processedWithErrors() > ingestionSchema.getTuningConfig().getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task...", e); - } - } - private static boolean exceedMaxRowsInSegment(int numRowsInSegment, IndexTuningConfig indexTuningConfig) { // maxRowsInSegment should be null if numShards is set in indexTuningConfig diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java index e5afb68d994f..c9588374b24a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java @@ -141,6 +141,16 @@ public SinglePhaseParallelIndexSupervisorTask( this.indexingServiceClient = indexingServiceClient; this.chatHandlerProvider = chatHandlerProvider; this.authorizerMapper = authorizerMapper; + + if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { + log.warn("maxSavedParseExceptions is not supported yet"); + } + if (ingestionSchema.getTuningConfig().getMaxParseExceptions() > 0) { + log.warn("maxParseExceptions is not supported yet"); + } + if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { + log.warn("logParseExceptions is not supported yet"); + } } @Override From fd7dda3ef0bcb5c42630ff947cbdf1ce27ae2191 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 13 Apr 2018 12:56:18 -0700 Subject: [PATCH 14/47] working --- .../io/druid/data/input/FiniteFirehoseFactory.java | 4 ++-- .../input/impl/AbstractTextFilesFirehoseFactory.java | 10 +++------- .../task/SinglePhaseParallelIndexSupervisorTask.java | 11 +++++------ .../SinglePhaseParallelIndexSupervisorTaskTest.java | 8 ++++---- 4 files changed, 14 insertions(+), 19 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java index e9022d7becad..9e4bd7fba40e 100644 --- a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java @@ -23,7 +23,7 @@ import io.druid.data.input.impl.InputRowParser; import java.io.IOException; -import java.util.Iterator; +import java.util.stream.Stream; /** * {@link FirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is @@ -48,7 +48,7 @@ default boolean isSplittable() * Returns an iterator of {@link InputSplit}s. */ @JsonIgnore - Iterator> getSplits() throws IOException; + Stream> getSplits() throws IOException; /** * Returns number of splits returned by {@link #getSplits()}. diff --git a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java index 01b264c07122..d4f8a37ff637 100644 --- a/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/impl/AbstractTextFilesFirehoseFactory.java @@ -37,7 +37,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; -import java.util.stream.Collectors; +import java.util.stream.Stream; /** * This is an abstract class for firehose factory for making firehoses reading text files. @@ -103,14 +103,10 @@ public List getObjects() } @Override - public Iterator> getSplits() throws IOException + public Stream> getSplits() throws IOException { initializeObjectsIfNeeded(); - return getObjects() - .stream() - .map(InputSplit::new) - .collect(Collectors.toList()) - .iterator(); + return getObjects().stream().map(InputSplit::new); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java index c9588374b24a..5d55eb0b58c9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java @@ -26,8 +26,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -80,6 +78,7 @@ import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import java.util.stream.Stream; /** * SinglePhaseParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing. This is @@ -223,7 +222,7 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception ); chatHandlerProvider.register(getId(), this, false); - final Iterator subTaskSpecIterator = subTaskSpecIterator(); + final Iterator subTaskSpecIterator = subTaskSpecIterator().iterator(); final int numTotalTasks = baseFirehoseFactory.getNumSplits(); final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs(); TaskState state = TaskState.FAILED; @@ -468,9 +467,9 @@ int getAndIncreaseNextSpecId() } @VisibleForTesting - Iterator subTaskSpecIterator() throws IOException + Stream subTaskSpecIterator() throws IOException { - return Iterators.transform(baseFirehoseFactory.getSplits(), this::newTaskSpec); + return baseFirehoseFactory.getSplits().map(this::newTaskSpec); } @VisibleForTesting @@ -496,7 +495,7 @@ private static List getSplitsIfSplittable(FirehoseFactory firehoseFa { if (firehoseFactory instanceof FiniteFirehoseFactory) { final FiniteFirehoseFactory finiteFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; - return Lists.newArrayList(finiteFirehoseFactory.getSplits()); + return finiteFirehoseFactory.getSplits().collect(Collectors.toList()); } else { throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName()); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java index aea25e9e1ae7..4bcdad224bf2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java @@ -21,7 +21,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; @@ -83,6 +82,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.stream.Stream; public class SinglePhaseParallelIndexSupervisorTaskTest extends IngestionTestBase { @@ -158,7 +158,7 @@ public void testIsReady() throws Exception prepareTaskForLocking(task); Assert.assertTrue(task.isReady(actionClient)); - final Iterator subTaskSpecIterator = task.subTaskSpecIterator(); + final Iterator subTaskSpecIterator = task.subTaskSpecIterator().iterator(); while (subTaskSpecIterator.hasNext()) { final SinglePhaseParallelIndexSubTaskSpec spec = subTaskSpecIterator.next(); @@ -322,12 +322,12 @@ private static class TestSinglePhaseParallelIndexSupervisorTask extends SinglePh } @Override - Iterator subTaskSpecIterator() throws IOException + Stream subTaskSpecIterator() throws IOException { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() .getFirehoseFactory(); - return Iterators.transform(baseFirehoseFactory.getSplits(), split -> { + return baseFirehoseFactory.getSplits().map(split -> { try { // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id Thread.sleep(10); From b142a309d37ae7ae74455c435aac94da8117eb92 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 13 Apr 2018 14:48:44 -0700 Subject: [PATCH 15/47] Fix split of static-s3 firehose --- .../firehose/s3/StaticS3FirehoseFactory.java | 7 +- .../s3/StaticS3FirehoseFactoryTest.java | 66 ++++++++++++++++++- 2 files changed, 69 insertions(+), 4 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 73d6843f0697..8583f32bd3f1 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -39,6 +39,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.IOE; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.storage.s3.S3Utils; @@ -267,11 +268,11 @@ public FiniteFirehoseFactory withSplit(In final String path = split.get().getKey(); final URI splitUri; if (authority.endsWith("/") && path.startsWith("/")) { - splitUri = URI.create(authority + path.substring(1)); + splitUri = URI.create(StringUtils.format("s3://%s%s", authority, path.substring(1))); } else if (!authority.endsWith("/") && !path.startsWith("/")) { - splitUri = URI.create(authority + "/" + path); + splitUri = URI.create(StringUtils.format("s3://%s/%s", authority, path)); } else { - splitUri = URI.create(authority + path); + splitUri = URI.create(StringUtils.format("s3://%s%s", authority, path)); } return new StaticS3FirehoseFactory( s3Client, diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index e60210126943..75ee66d59877 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -21,6 +21,9 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.ListObjectsV2Request; +import com.amazonaws.services.s3.model.ListObjectsV2Result; +import com.amazonaws.services.s3.model.S3ObjectSummary; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.Module; @@ -33,19 +36,26 @@ import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Provides; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.impl.StringInputRowParser; import io.druid.initialization.DruidModule; +import io.druid.storage.s3.S3Utils; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; +import java.io.IOException; import java.net.URI; import java.util.Arrays; +import java.util.Comparator; import java.util.List; +import java.util.stream.Collectors; /** */ public class StaticS3FirehoseFactoryTest { - private static final AmazonS3Client SERVICE = new AmazonS3Client(); + private static final AmazonS3Client SERVICE = EasyMock.createNiceMock(AmazonS3Client.class); @Test public void testSerde() throws Exception @@ -76,6 +86,60 @@ public void testSerde() throws Exception Assert.assertEquals(factory, outputFact); } + @Test + public void testWithSplit() throws IOException + { + final List uris = Arrays.asList( + URI.create("s3://foo/bar/file.gz"), + URI.create("s3://bar/foo/file2.gz") + ); + uris.sort(Comparator.comparing(URI::toString)); + + uris.forEach(StaticS3FirehoseFactoryTest::addExpectedObjject); + EasyMock.replay(SERVICE); + + final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory( + SERVICE, + uris, + null, + 2048L, + 1024L, + 512L, + 100L, + 5 + ); + final List> subFactories = factory + .getSplits() + .map(factory::withSplit) + .sorted(Comparator.comparing(eachFactory -> { + final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) eachFactory; + return staticS3FirehoseFactory.getUris().toString(); + })) + .collect(Collectors.toList()); + + Assert.assertEquals(uris.size(), subFactories.size()); + for (int i = 0; i < uris.size(); i++) { + final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) subFactories.get(i); + final List subFactoryUris = staticS3FirehoseFactory.getUris(); + Assert.assertEquals(1, subFactoryUris.size()); + Assert.assertEquals(uris.get(i), subFactoryUris.get(0)); + } + } + + private static void addExpectedObjject(URI uri) + { + final String s3Bucket = uri.getAuthority(); + final String key = S3Utils.extractS3Key(uri); + final S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setBucketName(s3Bucket); + objectSummary.setKey(key); + final ListObjectsV2Result result = new ListObjectsV2Result(); + result.setBucketName(s3Bucket); + result.setKeyCount(1); + result.getObjectSummaries().add(objectSummary); + EasyMock.expect(SERVICE.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))).andReturn(result); + } + private static ObjectMapper createObjectMapper(DruidModule baseModule) { final Injector injector = Guice.createInjector( From 1fe0a056255edd5201fc973310538f3ca1b16e51 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 23 Apr 2018 13:14:04 -0700 Subject: [PATCH 16/47] Add endpoints to supervisor task and a unit test for endpoints --- .../java/io/druid/data/input/InputSplit.java | 7 +- .../SinglePhaseParallelIndexSubTaskSpec.java | 11 +- ...inglePhaseParallelIndexSupervisorTask.java | 315 +++++++- .../indexing/common/task/SubTaskSpec.java | 20 +- .../indexing/common/task/TaskMonitor.java | 322 ++++++-- .../overlord/http/OverlordResource.java | 106 ++- .../overlord/http/TaskStatusResponse.java | 6 +- ...ePhaseParallelIndexSupervisorTaskTest.java | 329 +++++++++ ...rallelIndexSupervisorTaskResourceTest.java | 686 ++++++++++++++++++ ...ePhaseParallelIndexSupervisorTaskTest.java | 289 +------- .../indexing/common/task/TaskMonitorTest.java | 36 +- .../indexing/HttpIndexingServiceClient.java | 17 +- .../indexing/IndexingServiceClient.java | 1 - .../client/indexing/TaskStatusResponse.java | 6 +- .../indexing/NoopIndexingServiceClient.java | 3 +- 15 files changed, 1714 insertions(+), 440 deletions(-) create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java diff --git a/api/src/main/java/io/druid/data/input/InputSplit.java b/api/src/main/java/io/druid/data/input/InputSplit.java index 70d7223ba4df..544964a60602 100644 --- a/api/src/main/java/io/druid/data/input/InputSplit.java +++ b/api/src/main/java/io/druid/data/input/InputSplit.java @@ -19,6 +19,9 @@ package io.druid.data.input; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + /** * Input unit for distributed batch ingestion. Used in {@link FiniteFirehoseFactory}. */ @@ -26,11 +29,13 @@ public class InputSplit { private final T split; - public InputSplit(T split) + @JsonCreator + public InputSplit(@JsonProperty("split") T split) { this.split = split; } + @JsonProperty("split") public T get() { return split; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java index 26fbf3fae880..21129b068a35 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java @@ -19,24 +19,31 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.input.InputSplit; + import java.util.Map; class SinglePhaseParallelIndexSubTaskSpec extends SubTaskSpec { private final SinglePhaseParallelIndexIngestionSpec ingestionSpec; + @JsonCreator SinglePhaseParallelIndexSubTaskSpec( String id, String groupId, String supervisorTaskId, SinglePhaseParallelIndexIngestionSpec indexIngestionSpec, - Map context + Map context, + InputSplit inputSplit ) { - super(id, groupId, supervisorTaskId, context); + super(id, groupId, supervisorTaskId, context, inputSplit); this.ingestionSpec = indexIngestionSpec; } + @JsonProperty public SinglePhaseParallelIndexIngestionSpec getIngestionSpec() { return ingestionSpec; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java index 5d55eb0b58c9..111003903a0a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java @@ -26,6 +26,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -43,10 +44,11 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.common.task.TaskMonitor.MonitorEntry; import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; +import io.druid.indexing.common.task.TaskMonitor.TaskHistory; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.RE; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; @@ -62,11 +64,19 @@ import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; +import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -103,10 +113,12 @@ public class SinglePhaseParallelIndexSupervisorTask extends AbstractTask impleme private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); - // subtaskId -> report + // subTaskId -> report private final ConcurrentMap segmentsMap = new ConcurrentHashMap<>(); private volatile boolean stopped; + private volatile TaskMonitor taskMonitor; + private int nextSpecId = 0; @JsonCreator @@ -211,11 +223,6 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception private TaskStatus runParallel(TaskToolbox toolbox) throws Exception { - final TaskMonitor taskMonitor = new TaskMonitor<>( - Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"), - ingestionSchema.getTuningConfig().getMaxRetry() - ); - log.info( "Found chat handler of class[%s]", Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName() @@ -223,13 +230,15 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception chatHandlerProvider.register(getId(), this, false); final Iterator subTaskSpecIterator = subTaskSpecIterator().iterator(); - final int numTotalTasks = baseFirehoseFactory.getNumSplits(); final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs(); - TaskState state = TaskState.FAILED; - log.info("Total number of tasks is [%d]", numTotalTasks); + taskMonitor = new TaskMonitor<>( + Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"), + ingestionSchema.getTuningConfig().getMaxRetry(), + baseFirehoseFactory.getNumSplits() + ); + TaskState state = TaskState.RUNNING; - int numCompleteTasks = 0; taskMonitor.start(taskStatusCheckingPeriod); try { @@ -250,7 +259,6 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception final TaskState completeState = taskCompleteEvent.getLastState(); switch (completeState) { case SUCCESS: - numCompleteTasks++; final TaskStatusPlus completeStatus = taskCompleteEvent.getLastStatus(); if (completeStatus == null) { throw new ISE("Last status of complete task is missing!"); @@ -260,11 +268,11 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception throw new ISE("Missing reports from task[%s]!", completeStatus.getId()); } - log.info("[%d/%d] tasks succeeded", numCompleteTasks, numTotalTasks); if (!subTaskSpecIterator.hasNext()) { + // We have no more subTasks to run if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) { stopped = true; - if (numCompleteTasks == numTotalTasks) { + if (taskMonitor.isSucceeded()) { // Publishing all segments reported so far publish(toolbox); @@ -273,14 +281,19 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception } else { // Failed throw new ISE( - "Expected to complete [%d] tasks, but we got [%d] tasks", - numTotalTasks, - numCompleteTasks + "Expected for [%d] tasks to succeed, but we got [%d] succeeded tasks and [%d] failed tasks", + taskMonitor.getExpectedNumSucceededTasks(), + taskMonitor.getNumSucceededTasks(), + taskMonitor.getNumFailedTasks() ); } } } else if (taskMonitor.getNumRunningTasks() < maxNumTasks) { + // We have more subTasks to run submitNewTask(taskMonitor, subTaskSpecIterator.next()); + } else { + // We have more subTasks to run, but don't have enough available task slots + // do nothing } break; case FAILED: @@ -346,6 +359,8 @@ private TaskStatus runSequential(TaskToolbox toolbox) throws Exception ).run(toolbox); } + // Internal API for collecting reports from subTasks + @POST @Path("/report") @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE) @@ -428,6 +443,7 @@ private void submitNewTask( SinglePhaseParallelIndexSubTaskSpec spec ) { + log.info("Submit a new task for spec[%s] and inputSplit[%s]", spec.getId(), spec.getInputSplit()); final ListenableFuture> future = taskMonitor.submit(spec); Futures.addCallback( future, @@ -444,24 +460,15 @@ public void onSuccess(SubTaskCompleteEvent comp public void onFailure(Throwable t) { // this callback is called only when there were some problems in TaskMonitor. - try { - log.error( - t, - "Error while running a task for inputSplits[%s]", - getSplitsIfSplittable(spec.getIngestionSpec().getIOConfig().getFirehoseFactory()) - ); - } - catch (IOException e) { - t.addSuppressed(new RE(e, "Error while getting splits for error logging")); - } - taskCompleteEvents.offer(new SubTaskCompleteEvent<>(spec, TaskState.FAILED, null)); + log.error(t, "Error while running a task for subTaskSpec[%s]", spec); + taskCompleteEvents.offer(SubTaskCompleteEvent.fail(spec, t)); } } ); } @VisibleForTesting - int getAndIncreaseNextSpecId() + int getAndIncrementNextSpecId() { return nextSpecId++; } @@ -476,7 +483,7 @@ Stream subTaskSpecIterator() throws IOExcep SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) { return new SinglePhaseParallelIndexSubTaskSpec( - getId() + "_" + getAndIncreaseNextSpecId(), + getId() + "_" + getAndIncrementNextSpecId(), getGroupId(), getId(), new SinglePhaseParallelIndexIngestionSpec( @@ -487,7 +494,8 @@ SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) ), ingestionSchema.getTuningConfig() ), - getContext() + getContext(), + split ); } @@ -500,4 +508,249 @@ private static List getSplitsIfSplittable(FirehoseFactory firehoseFa throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName()); } } + + // External APIs to get running status + + @GET + @Path("isRunningInParallel") + @Produces(MediaType.APPLICATION_JSON) + public Response isRunningInParallel(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + return okResponse("isRunningInParallel", baseFirehoseFactory.isSplittable()); + } + + @GET + @Path("/numRunningTasks") + @Produces(MediaType.APPLICATION_JSON) + public Response getNumRunningTasks(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + final int numRunningTasks = taskMonitor == null ? 0 : taskMonitor.getNumRunningTasks(); + return okResponse("numRunningTasks", numRunningTasks); + } + + @GET + @Path("/numSucceededTasks") + @Produces(MediaType.APPLICATION_JSON) + public Response getNumSucceededTasks(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + final int numSucceededTasks = taskMonitor == null ? 0 : taskMonitor.getNumSucceededTasks(); + return okResponse("numSucceededTasks", numSucceededTasks); + } + + @GET + @Path("/numFailedTasks") + @Produces(MediaType.APPLICATION_JSON) + public Response getNumFailedTasks(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + final int numFailedTasks = taskMonitor == null ? 0 : taskMonitor.getNumFailedTasks(); + return okResponse("numFailedTasks", numFailedTasks); + } + + @GET + @Path("/numCompleteTasks") + @Produces(MediaType.APPLICATION_JSON) + public Response getNumCompleteTasks(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + final int numCompleteTasks = taskMonitor == null ? 0 : taskMonitor.getNumCompleteTasks(); + return okResponse("numCompleteTasks", numCompleteTasks); + } + + @GET + @Path("/expectedNumSucceededTasks") + @Produces(MediaType.APPLICATION_JSON) + public Response getExpectedNumSucceededTasks(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + final int expectedNumSucceededTasks = taskMonitor == null ? 0 : taskMonitor.getExpectedNumSucceededTasks(); + return okResponse("expectedNumSucceededTasks", expectedNumSucceededTasks); + } + + @GET + @Path("/runningSubTasks") + @Produces(MediaType.APPLICATION_JSON) + public Response getRunningTasks(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + final Set runningTasks = taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds(); + return okResponse("runningSubTasks", runningTasks); + } + + @GET + @Path("/subTaskSpecs") + @Produces(MediaType.APPLICATION_JSON) + public Response getSubTaskSpecs(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); + final List> completeSubTaskSpecs = taskMonitor + .getCompleteSubTaskSpecs(); + // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and + // completeSubTaskSpecs. + final Map> subTaskSpecMap = new HashMap<>( + runningSubTaskSpecs.size() + completeSubTaskSpecs.size() + ); + runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); + completeSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); + return okResponse("subTaskSpecs", new ArrayList<>(subTaskSpecMap.values())); + } + + @GET + @Path("/runningSubTaskSpecs") + @Produces(MediaType.APPLICATION_JSON) + public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + return okResponse("runningSubTaskSpecs", taskMonitor.getRunningSubTaskSpecs()); + } + + @GET + @Path("/completeSubTaskSpecs") + @Produces(MediaType.APPLICATION_JSON) + public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + return okResponse("completeSubTaskSpecs", taskMonitor.getCompleteSubTaskSpecs()); + } + + @GET + @Path("/subTaskSpec") + @Produces(MediaType.APPLICATION_JSON) + public Response getSubTaskSpec(@QueryParam("id") String id, @Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after + // adding them to taskHistory. + final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); + final SubTaskSpec subTaskSpec; + + if (monitorEntry != null) { + subTaskSpec = monitorEntry.getSpec(); + } else { + if (taskHistory != null) { + subTaskSpec = taskHistory.getSpec(); + } else { + subTaskSpec = null; + } + } + + if (subTaskSpec == null) { + return Response.status(Status.NOT_FOUND).build(); + } else { + return Response.ok(subTaskSpec).build(); + } + } + + @GET + @Path("/subTaskState") + @Produces(MediaType.APPLICATION_JSON) + public Response getSubTaskState(@QueryParam("id") String id, @Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (taskMonitor == null) { + return Response.status(Status.NOT_FOUND).build(); + } else { + // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after + // adding them to taskHistory. + final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); + + final SubTaskStateResponse subTaskStateResponse; + + if (monitorEntry != null) { + subTaskStateResponse = new SubTaskStateResponse( + (SinglePhaseParallelIndexSubTaskSpec) monitorEntry.getSpec(), + monitorEntry.getRunningStatus(), + monitorEntry.getTaskHistory() + ); + } else { + if (taskHistory != null && !taskHistory.isEmpty()) { + subTaskStateResponse = new SubTaskStateResponse( + (SinglePhaseParallelIndexSubTaskSpec) taskHistory.getSpec(), + null, + taskHistory.getAttemptHistory() + ); + } else { + subTaskStateResponse = null; + } + } + + if (subTaskStateResponse == null) { + return Response.status(Status.NOT_FOUND).build(); + } else { + return Response.ok(subTaskStateResponse).build(); + } + } + } + + @GET + @Path("/completeSubTaskSpecAttemptHistory") + @Produces(MediaType.APPLICATION_JSON) + public Response getCompleteSubTaskSpecAttemptHistory( + @QueryParam("id") String id, + @Context final HttpServletRequest req + ) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (taskMonitor == null) { + return Response.status(Status.NOT_FOUND).build(); + } else { + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); + + if (taskHistory == null) { + return Response.status(Status.NOT_FOUND).build(); + } else { + return Response.ok(taskHistory.getAttemptHistory()).build(); + } + } + } + + private static Response okResponse(String key, Object val) + { + return Response.ok(ImmutableMap.of(key, val)).build(); + } + + static class SubTaskStateResponse + { + private final SinglePhaseParallelIndexSubTaskSpec spec; + @Nullable + private final TaskStatusPlus currentStatus; + private final List taskHistory; + + @JsonCreator + public SubTaskStateResponse( + @JsonProperty("spec") SinglePhaseParallelIndexSubTaskSpec spec, + @JsonProperty("currentStatus") @Nullable TaskStatusPlus currentStatus, + @JsonProperty("taskHistory") List taskHistory + ) + { + this.spec = spec; + this.currentStatus = currentStatus; + this.taskHistory = taskHistory; + } + + @JsonProperty + public SinglePhaseParallelIndexSubTaskSpec getSpec() + { + return spec; + } + + @JsonProperty + @Nullable + public TaskStatusPlus getCurrentStatus() + { + return currentStatus; + } + + @JsonProperty + public List getTaskHistory() + { + return taskHistory; + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java index 2472d25c81a3..0d9cb76fee10 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java @@ -19,6 +19,10 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.input.InputSplit; + import java.util.Map; public abstract class SubTaskSpec @@ -27,39 +31,53 @@ public abstract class SubTaskSpec private final String groupId; private final String supervisorTaskId; private final Map context; + private final InputSplit inputSplit; + @JsonCreator public SubTaskSpec( String id, String groupId, String supervisorTaskId, - Map context + Map context, + InputSplit inputSplit ) { this.id = id; this.groupId = groupId; this.supervisorTaskId = supervisorTaskId; this.context = context; + this.inputSplit = inputSplit; } + @JsonProperty public String getId() { return id; } + @JsonProperty public String getGroupId() { return groupId; } + @JsonProperty public String getSupervisorTaskId() { return supervisorTaskId; } + @JsonProperty public Map getContext() { return context; } + @JsonProperty + public InputSplit getInputSplit() + { + return inputSplit; + } + public abstract T newSubTask(int numAttempts); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index 76dd08c13baa..81ce79dd946e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -20,6 +20,7 @@ package io.druid.indexing.common.task; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; @@ -32,15 +33,16 @@ import io.druid.java.util.common.logger.Logger; import javax.annotation.Nullable; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map.Entry; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; /** * Responsible for submitting tasks, monitoring task statuses, resubmitting failed tasks, and returning the final task @@ -52,28 +54,50 @@ public class TaskMonitor private final ScheduledExecutorService taskStatusChecker = Execs.scheduledSingleThreaded(("task-monitor-%d")); - // taskId -> monitorEntry + /** + * A map of subTaskSpecId to {@link MonitorEntry}. This map stores the state of running {@link SubTaskSpec}s. This is + * read in {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker} and updated in {@link #submit} + * and {@link #retry}. This can also be read by calling {@link #getRunningTaskMonitorEntory}, + * {@link #getRunningTaskIds}, and {@link #getRunningSubTaskSpecs}. + */ private final ConcurrentMap runningTasks = new ConcurrentHashMap<>(); - // overlord client - private final AtomicInteger numRunningTasks = new AtomicInteger(); + /** + * A map of subTaskSpecId to {@link TaskHistory}. This map stores the history of complete {@link SubTaskSpec}s + * whether their final state is succeeded or failed. This is updated in {@link MonitorEntry#setLastStatus} which is + * called by the {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker} and can be + * read by outside of this class. + */ + private final ConcurrentMap> taskHistories = new ConcurrentHashMap<>(); + + // lock for updating numRunningTasks, numSucceededTasks, and numFailedTasks + private final Object taskCountLock = new Object(); + // overlord client private final IndexingServiceClient indexingServiceClient; private final int maxRetry; + private final int expectedNumSucceededTasks; + + private int numRunningTasks; + private int numSucceededTasks; + private int numFailedTasks; private volatile boolean running = false; - TaskMonitor(IndexingServiceClient indexingServiceClient, int maxRetry) + TaskMonitor(IndexingServiceClient indexingServiceClient, int maxRetry, int expectedNumSucceededTasks) { this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"); this.maxRetry = maxRetry; + this.expectedNumSucceededTasks = expectedNumSucceededTasks; + + log.info("TaskMonitor is initialized with expectedNumSucceededTasks[%d]", expectedNumSucceededTasks); } public void start(long taskStatusCheckingPeriod) { running = true; log.info("Starting taskMonitor"); - // NOTE: This polling can be improved to event-driven processing by registering TaskRunnerListener to TaskRunner. + // NOTE: This polling can be improved to event-driven pushing by registering TaskRunnerListener to TaskRunner. // That listener should be able to send the events reported to TaskRunner to this TaskMonitor. taskStatusChecker.scheduleAtFixedRate( () -> { @@ -81,41 +105,47 @@ public void start(long taskStatusCheckingPeriod) final Iterator> iterator = runningTasks.entrySet().iterator(); while (iterator.hasNext()) { final Entry entry = iterator.next(); - final String taskId = entry.getKey(); + final String specId = entry.getKey(); final MonitorEntry monitorEntry = entry.getValue(); + final String taskId = monitorEntry.runningTask.getId(); final TaskStatusResponse taskStatusResponse = indexingServiceClient.getTaskStatus(taskId); - if (taskStatusResponse != null) { - final TaskStatusPlus taskStatus = taskStatusResponse.getStatus(); - switch (taskStatus.getState()) { + final TaskStatusPlus taskStatus = taskStatusResponse.getStatus(); + if (taskStatus != null) { + switch (Preconditions.checkNotNull(taskStatus.getState(), "taskState")) { case SUCCESS: - numRunningTasks.decrementAndGet(); - iterator.remove(); + incrementNumSucceededTasks(); + // Remote the current entry after updating taskHistories to make sure that task history + // exists either runningTasks or taskHistories. monitorEntry.setLastStatus(taskStatus); + iterator.remove(); break; case FAILED: - numRunningTasks.decrementAndGet(); - iterator.remove(); + incrementNumFailedTasks(); log.warn("task[%s] failed!", taskId); if (monitorEntry.numTries() < maxRetry) { log.info( - "We still have chnaces[%d/%d] to complete.", + "We still have chances[%d/%d] to complete for spec[%s].", monitorEntry.numTries(), - maxRetry + maxRetry, + monitorEntry.spec.getId() ); - retry(monitorEntry, taskStatus); + retry(specId, monitorEntry, taskStatus); } else { log.error( "spec[%s] failed after [%d] tries", monitorEntry.spec.getId(), monitorEntry.numTries() ); + // Remote the current entry after updating taskHistories to make sure that task history + // exists either runningTasks or taskHistories. monitorEntry.setLastStatus(taskStatus); + iterator.remove(); } break; case RUNNING: - // do nothing + monitorEntry.updateStatus(taskStatus); break; default: throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getState(), taskId); @@ -125,6 +155,7 @@ public void start(long taskStatusCheckingPeriod) } catch (Throwable t) { log.error(t, "Error while monitoring"); + throw t; } }, taskStatusCheckingPeriod, @@ -146,81 +177,198 @@ public ListenableFuture> submit(SubTaskSpec spec) return Futures.immediateFailedFuture(new ISE("TaskMonitore is not running")); } final T task = spec.newSubTask(0); - log.info("Submitting a new task[%s]", task.getId()); + log.info("Submitting a new task[%s] for spec[%s]", task.getId(), spec.getId()); indexingServiceClient.runTask(task); - numRunningTasks.incrementAndGet(); + incrementNumRunningTasks(); final SettableFuture> taskFuture = SettableFuture.create(); - runningTasks.put(task.getId(), new MonitorEntry(spec, task, taskFuture)); + runningTasks.put( + spec.getId(), + new MonitorEntry(spec, task, indexingServiceClient.getTaskStatus(task.getId()).getStatus(), taskFuture) + ); return taskFuture; } - private void retry(MonitorEntry monitorEntry, TaskStatusPlus lastFailedTaskStatus) + /** + * Submit a retry task for a failed spec. This method should be called inside of the + * {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker}. + */ + private void retry(String subTaskSpecId, MonitorEntry monitorEntry, TaskStatusPlus lastFailedTaskStatus) { if (running) { final SubTaskSpec spec = monitorEntry.spec; final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1); log.info("Submitting a new task[%s] for retrying spec[%s]", task.getId(), spec.getId()); indexingServiceClient.runTask(task); - numRunningTasks.incrementAndGet(); + incrementNumRunningTasks(); runningTasks.put( - task.getId(), - monitorEntry.withNewRunningTask(task, lastFailedTaskStatus) + subTaskSpecId, + monitorEntry.withNewRunningTask( + task, + indexingServiceClient.getTaskStatus(task.getId()).getStatus(), + lastFailedTaskStatus + ) ); } } - public void killAll() + /** + * This method should be called after {@link #stop()} to make sure no additional tasks are submitted. + */ + void killAll() { - runningTasks.keySet().forEach(taskId -> { + runningTasks.values().forEach(entry -> { + final String taskId = entry.runningTask.getId(); log.info("Request to kill subtask[%s]", taskId); indexingServiceClient.killTask(taskId); }); runningTasks.clear(); } - public int getNumRunningTasks() + void incrementNumRunningTasks() + { + synchronized (taskCountLock) { + numRunningTasks++; + } + } + + void incrementNumSucceededTasks() + { + synchronized (taskCountLock) { + numRunningTasks--; + numSucceededTasks++; + log.info("[%d/%d] tasks succeeded", numSucceededTasks, expectedNumSucceededTasks); + } + } + + void incrementNumFailedTasks() + { + synchronized (taskCountLock) { + numRunningTasks--; + numFailedTasks++; + } + } + + boolean isSucceeded() + { + synchronized (taskCountLock) { + return numSucceededTasks == expectedNumSucceededTasks; + } + } + + int getExpectedNumSucceededTasks() + { + return expectedNumSucceededTasks; + } + + int getNumRunningTasks() + { + synchronized (taskCountLock) { + return numRunningTasks; + } + } + + int getNumCompleteTasks() + { + synchronized (taskCountLock) { + return numSucceededTasks + numFailedTasks; + } + } + + int getNumSucceededTasks() + { + synchronized (taskCountLock) { + return numSucceededTasks; + } + } + + int getNumFailedTasks() + { + synchronized (taskCountLock) { + return numFailedTasks; + } + } + + Set getRunningTaskIds() + { + return runningTasks.values().stream().map(entry -> entry.runningTask.getId()).collect(Collectors.toSet()); + } + + List> getRunningSubTaskSpecs() + { + return runningTasks.values().stream().map(monitorEntry -> monitorEntry.spec).collect(Collectors.toList()); + } + + @Nullable + MonitorEntry getRunningTaskMonitorEntory(String subTaskSpecId) + { + return runningTasks.values() + .stream() + .filter(monitorEntry -> monitorEntry.spec.getId().equals(subTaskSpecId)) + .findFirst() + .orElse(null); + } + + List> getCompleteSubTaskSpecs() + { + return taskHistories.values().stream().map(history -> history.spec).collect(Collectors.toList()); + } + + @Nullable + TaskHistory getCompleteSubTaskSpecHistory(String subTaskSpecId) { - return numRunningTasks.intValue(); + return taskHistories.get(subTaskSpecId); } - private class MonitorEntry + class MonitorEntry { private final SubTaskSpec spec; private final T runningTask; - private final List taskHistory; + // old tasks to recent tasks. running task is not included + private final CopyOnWriteArrayList taskHistory; private final SettableFuture> completeEventFuture; + /** + * This variable is updated inside of the {@link java.util.concurrent.Callable} executed by + * {@link #taskStatusChecker}, and can be read by calling {@link #getRunningStatus}. + */ + @Nullable + private volatile TaskStatusPlus runningStatus; + MonitorEntry( SubTaskSpec spec, T runningTask, + @Nullable TaskStatusPlus runningStatus, SettableFuture> completeEventFuture ) { - this(spec, runningTask, new ArrayList<>(), completeEventFuture); + this(spec, runningTask, runningStatus, new CopyOnWriteArrayList<>(), completeEventFuture); } private MonitorEntry( SubTaskSpec spec, T runningTask, - List taskHistory, + @Nullable TaskStatusPlus runningStatus, + CopyOnWriteArrayList taskHistory, SettableFuture> completeEventFuture ) { this.spec = spec; this.runningTask = runningTask; + this.runningStatus = runningStatus; this.taskHistory = taskHistory; this.completeEventFuture = completeEventFuture; } - MonitorEntry withNewRunningTask(T newTask, TaskStatusPlus statusOfLastTask) + MonitorEntry withNewRunningTask(T newTask, @Nullable TaskStatusPlus newStatus, TaskStatusPlus statusOfLastTask) { taskHistory.add(statusOfLastTask); return new MonitorEntry( spec, newTask, + newStatus, taskHistory, completeEventFuture ); @@ -228,7 +376,19 @@ MonitorEntry withNewRunningTask(T newTask, TaskStatusPlus statusOfLastTask) int numTries() { - return taskHistory.size() + 1; // count runningTask. this is valid only until setLastStatus() is called + return taskHistory.size() + 1; // count runningTask as well + } + + void updateStatus(TaskStatusPlus statusPlus) + { + if (!runningTask.getId().equals(statusPlus.getId())) { + throw new ISE( + "Task id[%s] of lastStatus is different from the running task[%s]", + statusPlus.getId(), + runningTask.getId() + ); + } + this.runningStatus = statusPlus; } void setLastStatus(TaskStatusPlus lastStatus) @@ -241,27 +401,91 @@ void setLastStatus(TaskStatusPlus lastStatus) ); } + this.runningStatus = lastStatus; taskHistory.add(lastStatus); - completeEventFuture.set(new SubTaskCompleteEvent<>(spec, lastStatus.getState(), taskHistory)); + taskHistories.put(spec.getId(), new TaskHistory<>(spec, taskHistory)); + completeEventFuture.set(SubTaskCompleteEvent.success(spec, lastStatus)); + } + + SubTaskSpec getSpec() + { + return spec; + } + + @Nullable + TaskStatusPlus getRunningStatus() + { + return runningStatus; + } + + List getTaskHistory() + { + return taskHistory; + } + } + + static class TaskHistory + { + private final SubTaskSpec spec; + private final List attemptHistory; // old to recent + + TaskHistory(SubTaskSpec spec, List attemptHistory) + { + attemptHistory.forEach(status -> { + Preconditions.checkState( + status.getState() == TaskState.SUCCESS || status.getState() == TaskState.FAILED, + "Complete tasks should be recorded, but the state of task[%s] is [%s]", + status.getId(), + status.getState() + ); + }); + this.spec = spec; + this.attemptHistory = ImmutableList.copyOf(attemptHistory); + } + + SubTaskSpec getSpec() + { + return spec; + } + + List getAttemptHistory() + { + return attemptHistory; + } + + boolean isEmpty() + { + return attemptHistory.isEmpty(); } } static class SubTaskCompleteEvent { private final SubTaskSpec spec; - private final TaskState lastState; @Nullable - private final List attemptHistory; + private final TaskStatusPlus lastStatus; + @Nullable + private final Throwable throwable; - SubTaskCompleteEvent( + static SubTaskCompleteEvent success(SubTaskSpec spec, TaskStatusPlus lastStatus) + { + return new SubTaskCompleteEvent<>(spec, Preconditions.checkNotNull(lastStatus, "lastStatus"), null); + } + + static SubTaskCompleteEvent fail(SubTaskSpec spec, Throwable t) + { + return new SubTaskCompleteEvent<>(spec, null, t); + } + + private SubTaskCompleteEvent( SubTaskSpec spec, - TaskState lastState, - @Nullable List attemptHistory + @Nullable TaskStatusPlus lastStatus, + @Nullable Throwable throwable ) { this.spec = Preconditions.checkNotNull(spec, "spec"); - this.lastState = Preconditions.checkNotNull(lastState, "lastState"); - this.attemptHistory = attemptHistory; + this.lastStatus = lastStatus; + this.throwable = throwable; } SubTaskSpec getSpec() @@ -271,19 +495,19 @@ SubTaskSpec getSpec() TaskState getLastState() { - return lastState; + return lastStatus == null ? TaskState.FAILED : lastStatus.getState(); } @Nullable - List getAttemptHistory() + TaskStatusPlus getLastStatus() { - return attemptHistory; + return lastStatus; } @Nullable - TaskStatusPlus getLastStatus() + Throwable getThrowable() { - return attemptHistory == null ? null : attemptHistory.get(attemptHistory.size() - 1); + return throwable; } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 86c48cbbc57f..622ff799595b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -237,65 +237,63 @@ public Response getTaskPayload(@PathParam("taskid") String taskid) @ResourceFilters(TaskResourceFilter.class) public Response getTaskStatus(@PathParam("taskid") String taskid) { - if (taskMaster.getTaskRunner().isPresent()) { + final Task task = taskStorageQueryAdapter.getTask(taskid).orNull(); + final TaskStatus taskStatus = taskStorageQueryAdapter.getStatus(taskid).orNull(); + final Pair createdDateAndDataSource = taskStorageQueryAdapter.getCreatedDateAndDataSource( + taskid + ); + final TaskStatusResponse response; + if (taskMaster.getTaskRunner().isPresent() && + task != null && + taskStatus != null && + createdDateAndDataSource != null) { final TaskRunner taskRunner = taskMaster.getTaskRunner().get(); - - final Task task = taskStorageQueryAdapter.getTask(taskid).orNull(); - final TaskStatus taskStatus = taskStorageQueryAdapter.getStatus(taskid).orNull(); - final Pair createdDateAndDataSource = taskStorageQueryAdapter.getCreatedDateAndDataSource( - taskid - ); - final TaskStatusResponse response; - if (task != null && taskStatus != null && createdDateAndDataSource != null) { - final TaskRunnerWorkItem workItem = taskRunner - .getKnownTasks() - .stream() - .filter(item -> item.getTaskId().equals(taskid)) - .findAny() - .orElse(null); - if (workItem != null) { - response = new TaskStatusResponse( - taskid, - new TaskStatusPlus( - taskid, - task.getType(), - createdDateAndDataSource.lhs, - workItem.getQueueInsertionTime(), - taskStatus.getStatusCode(), - taskStatus.getDuration(), - workItem.getLocation(), - createdDateAndDataSource.rhs, - null - ) - ); - } else { - response = new TaskStatusResponse( - taskid, - new TaskStatusPlus( - taskid, - task.getType(), - createdDateAndDataSource.lhs, - DateTimes.EPOCH, - taskStatus.getStatusCode(), - taskStatus.getDuration(), - TaskLocation.unknown(), - createdDateAndDataSource.rhs, - null - ) - ); - } + final TaskRunnerWorkItem workItem = taskRunner + .getKnownTasks() + .stream() + .filter(item -> item.getTaskId().equals(taskid)) + .findAny() + .orElse(null); + if (workItem != null) { + response = new TaskStatusResponse( + taskid, + new TaskStatusPlus( + taskid, + task.getType(), + createdDateAndDataSource.lhs, + workItem.getQueueInsertionTime(), + taskStatus.getStatusCode(), + taskStatus.getDuration(), + workItem.getLocation(), + createdDateAndDataSource.rhs, + null + ) + ); } else { - response = new TaskStatusResponse(taskid, null); + response = new TaskStatusResponse( + taskid, + new TaskStatusPlus( + taskid, + task.getType(), + createdDateAndDataSource.lhs, + DateTimes.EPOCH, + taskStatus.getStatusCode(), + taskStatus.getDuration(), + TaskLocation.unknown(), + createdDateAndDataSource.rhs, + null + ) + ); } - - final Response.Status status = response.getStatus() == null - ? Response.Status.NOT_FOUND - : Response.Status.OK; - - return Response.status(status).entity(response).build(); } else { - return Response.status(Response.Status.NOT_FOUND).entity("No tasks are running").build(); + response = new TaskStatusResponse(taskid, null); } + + final Response.Status status = response.getStatus() == null + ? Response.Status.NOT_FOUND + : Response.Status.OK; + + return Response.status(status).entity(response).build(); } @GET diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java index 5e413ce92733..595a2cdc843f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/TaskStatusResponse.java @@ -23,17 +23,19 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexer.TaskStatusPlus; +import javax.annotation.Nullable; import java.util.Objects; public class TaskStatusResponse { private final String task; // Task ID, named "task" in the JSONification of this class. + @Nullable private final TaskStatusPlus status; @JsonCreator public TaskStatusResponse( @JsonProperty("task") final String task, - @JsonProperty("status") final TaskStatusPlus status + @JsonProperty("status") @Nullable final TaskStatusPlus status ) { this.task = task; @@ -47,6 +49,7 @@ public String getTask() } @JsonProperty + @Nullable public TaskStatusPlus getStatus() { return status; @@ -69,7 +72,6 @@ public boolean equals(final Object o) @Override public int hashCode() { - return Objects.hash(task, status); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java new file mode 100644 index 000000000000..8be59a8bd501 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java @@ -0,0 +1,329 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.client.indexing.NoopIndexingServiceClient; +import io.druid.client.indexing.TaskStatusResponse; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.CSVParseSpec; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.ParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.indexer.TaskLocation; +import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.TaskInfoProvider; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.segment.loading.LocalDataSegmentPusher; +import io.druid.segment.loading.LocalDataSegmentPusherConfig; +import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import io.druid.server.security.AllowAllAuthorizer; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; +import io.druid.timeline.DataSegment; +import org.joda.time.Duration; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.stream.Stream; + +public class AbstractSinglePhaseParallelIndexSupervisorTaskTest extends IngestionTestBase +{ + static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( + new TimestampSpec( + "ts", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), + Lists.newArrayList(), + Lists.newArrayList() + ), + null, + Arrays.asList("ts", "dim", "val"), + false, + 0 + ); + + TaskActionClient actionClient; + LocalIndexingServiceClient indexingServiceClient; + TaskToolbox toolbox; + File localDeepStorage; + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + class LocalIndexingServiceClient extends NoopIndexingServiceClient + { + private final ConcurrentMap> tasks = new ConcurrentHashMap<>(); + private final ListeningExecutorService service = MoreExecutors.listeningDecorator( + Execs.multiThreaded(5, "parallel-index-single-phase-supervisor-task-test-%d") + ); + + @Override + public String runTask(Object taskObject) + { + final SinglePhaseParallelIndexSubTask subTask = (SinglePhaseParallelIndexSubTask) taskObject; + tasks.put(subTask.getId(), service.submit(() -> { + try { + final TaskToolbox toolbox = createTaskToolbox(subTask); + if (subTask.isReady(toolbox.getTaskActionClient())) { + return subTask.run(toolbox); + } else { + throw new ISE("task[%s] is not ready", subTask.getId()); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + })); + return subTask.getId(); + } + + @Override + public TaskStatusResponse getTaskStatus(String taskId) + { + final Future taskStatusFuture = tasks.get(taskId); + if (taskStatusFuture != null) { + try { + if (taskStatusFuture.isDone()) { + final TaskStatus taskStatus = taskStatusFuture.get(); + return new TaskStatusResponse( + taskId, + new TaskStatusPlus( + taskId, + "index_single_phase_sub", + DateTimes.EPOCH, + DateTimes.EPOCH, + taskStatus.getStatusCode(), + -1L, + TaskLocation.unknown(), + null, + null + ) + ); + } else { + return new TaskStatusResponse( + taskId, + new TaskStatusPlus( + taskId, + "index_single_phase_sub", + DateTimes.EPOCH, + DateTimes.EPOCH, + TaskState.RUNNING, + -1L, + TaskLocation.unknown(), + null, + null + ) + ); + } + } + catch (InterruptedException | ExecutionException e) { + // We don't have a way to pass this exception to the supervisorTask yet.. + // So, let's print it here. + System.err.println(Throwables.getStackTraceAsString(e)); + return new TaskStatusResponse( + taskId, + new TaskStatusPlus( + taskId, + "index_single_phase_sub", + DateTimes.EPOCH, + DateTimes.EPOCH, + TaskState.FAILED, + -1L, + TaskLocation.unknown(), + null, + null + ) + ); + } + } else { + return new TaskStatusResponse(taskId, null); + } + } + + @Override + public String killTask(String taskId) + { + final Future taskStatusFuture = tasks.remove(taskId); + if (taskStatusFuture != null) { + taskStatusFuture.cancel(true); + return taskId; + } else { + return null; + } + } + + void shutdown() + { + service.shutdownNow(); + } + } + + TaskToolbox createTaskToolbox(Task task) throws IOException + { + return new TaskToolbox( + null, + actionClient, + null, + new LocalDataSegmentPusher( + new LocalDataSegmentPusherConfig() + { + @Override + public File getStorageDirectory() + { + return localDeepStorage; + } + }, + getObjectMapper() + ), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + getObjectMapper(), + temporaryFolder.newFolder(task.getId()), + getIndexIO(), + null, + null, + getIndexMerger(), + null, + null, + null, + null, + new NoopTestTaskFileWriter() + ); + } + + static class TestSinglePhaseParallelIndexSupervisorTask extends SinglePhaseParallelIndexSupervisorTask + { + TestSinglePhaseParallelIndexSupervisorTask( + String id, + TaskResource taskResource, + SinglePhaseParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + super( + id, + taskResource, + ingestionSchema, + context, + indexingServiceClient, + new NoopChatHandlerProvider(), + new AuthorizerMapper(ImmutableMap.of()) + { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + } + ); + } + + @Override + Stream subTaskSpecIterator() throws IOException + { + final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() + .getIOConfig() + .getFirehoseFactory(); + return baseFirehoseFactory.getSplits().map(split -> { + try { + // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id + Thread.sleep(10); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + return newTaskSpec((InputSplit) split); + }); + } + } + + static class LocalSinglePhaseParallelIndexTaskClientFactory + implements IndexTaskClientFactory + { + private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + + LocalSinglePhaseParallelIndexTaskClientFactory(SinglePhaseParallelIndexSupervisorTask supervisorTask) + { + this.supervisorTask = supervisorTask; + } + + @Override + public SinglePhaseParallelIndexTaskClient build( + TaskInfoProvider taskInfoProvider, + String callerId, + int numThreads, + Duration httpTimeout, + long numRetries + ) + { + return new LocalSinglePhaseParallelIndexTaskClient(callerId, supervisorTask); + } + } + + static class LocalSinglePhaseParallelIndexTaskClient extends SinglePhaseParallelIndexTaskClient + { + private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + + public LocalSinglePhaseParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexSupervisorTask supervisorTask) + { + super(null, null, null, null, callerId, 0); + this.supervisorTask = supervisorTask; + } + + @Override + public void report(String supervisorTaskId, List pushedSegments) + { + supervisorTask.collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); + } + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java new file mode 100644 index 000000000000..9bf2370b2cb6 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java @@ -0,0 +1,686 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.data.input.FiniteFirehoseFactory; +import io.druid.data.input.InputSplit; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.indexer.TaskLocation; +import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SurrogateLockListAction; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.SubTaskStateResponse; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NumberedShardSpec; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.core.Response; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +public class SinglePhaseParallelIndexSupervisorTaskResourceTest + extends AbstractSinglePhaseParallelIndexSupervisorTaskTest +{ + private static final int NUM_SUB_TASKS = 10; + + // specId -> spec + private final ConcurrentMap subTaskSpecs = new ConcurrentHashMap<>(); + + // specId -> taskStatusPlus + private final ConcurrentMap runningSpecs = new ConcurrentHashMap<>(); + + // specId -> taskStatusPlus list + private final ConcurrentMap> taskHistories = new ConcurrentHashMap<>(); + + // taskId -> subTaskSpec + private final ConcurrentMap taskIdToSpec = new ConcurrentHashMap<>(); + + // taskId -> task + private final CopyOnWriteArrayList runningTasks = new CopyOnWriteArrayList<>(); + + private ExecutorService service; + + private TestSupervisorTask task; + + @Before + public void setup() throws IOException + { + service = Execs.singleThreaded("single-phase-parallel-index-supervisor-task-resource-test-%d"); + indexingServiceClient = new LocalIndexingServiceClient(); + localDeepStorage = temporaryFolder.newFolder("localStorage"); + } + + @After + public void teardown() + { + indexingServiceClient.shutdown(); + temporaryFolder.delete(); + service.shutdownNow(); + } + + @Test(timeout = 20000L) + public void testAPIs() throws Exception + { + task = newTask( + Intervals.of("2017/2018"), + new SinglePhaseParallelIndexIOConfig( + new TestFirehose(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())), + false + ) + ); + actionClient = createActionClient(task); + toolbox = createTaskToolbox(task); + + prepareTaskForLocking(task); + Assert.assertTrue(task.isReady(actionClient)); + final Future supervisorTaskFuture = service.submit(() -> task.run(toolbox)); + + // test isRunningInParallel + Response response = task.isRunningInParallel(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("isRunningInParallel", true), response.getEntity()); + + // test expectedNumSucceededTasks + response = task.getExpectedNumSucceededTasks(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("expectedNumSucceededTasks", NUM_SUB_TASKS), response.getEntity()); + + // Since taskMonitor works based on polling, it's hard to use a fancier way to check its state. + // We use polling to check the state of taskMonitor in this test. + while (getNumSubTasks("numRunningTasks", req -> task.getNumRunningTasks(req)) < NUM_SUB_TASKS) { + Thread.sleep(100); + } + + int succeededTasks = 0; + int failedTasks = 0; + checkState( + task, + succeededTasks, + failedTasks, + buildStateMap() + ); + + // numRunningTasks and numSucceededTasks after some successful subTasks + succeededTasks += 2; + for (int i = 0; i < succeededTasks; i++) { + runningTasks.get(0).setState(TaskState.SUCCESS); + } + + while (getNumSubTasks("numSucceededTasks", req -> task.getNumSucceededTasks(req)) < succeededTasks) { + Thread.sleep(100); + } + + checkState( + task, + succeededTasks, + failedTasks, + buildStateMap() + ); + + // numRunningTasks and numSucceededTasks after some failed subTasks + failedTasks += 3; + for (int i = 0; i < failedTasks; i++) { + runningTasks.get(0).setState(TaskState.FAILED); + } + + while (getNumSubTasks("numFailedTasks", req -> task.getNumFailedTasks(req)) < failedTasks) { + Thread.sleep(100); + } + + checkState( + task, + succeededTasks, + failedTasks, + buildStateMap() + ); + + // Wait for new tasks to be started + while (runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { + Thread.sleep(100); + } + + // Make sure only one subTask is running + succeededTasks += 7; + for (int i = 0; i < 7; i++) { + runningTasks.get(0).setState(TaskState.SUCCESS); + } + + while (getNumSubTasks("numSucceededTasks", req -> task.getNumSucceededTasks(req)) < succeededTasks) { + Thread.sleep(100); + } + + checkState( + task, + succeededTasks, + failedTasks, + buildStateMap() + ); + + Assert.assertEquals(1, runningSpecs.size()); + final String lastRunningSpecId = runningSpecs.keySet().iterator().next(); + final List taskHistory = taskHistories.get(lastRunningSpecId); + // This should be a failed task history because new tasks appear later in runningTasks. + Assert.assertEquals(1, taskHistory.size()); + + // Test one more failure + runningTasks.get(0).setState(TaskState.FAILED); + failedTasks++; + while (getNumSubTasks("numFailedTasks", req -> task.getNumFailedTasks(req)) < failedTasks) { + Thread.sleep(100); + } + + checkState( + task, + succeededTasks, + failedTasks, + buildStateMap() + ); + Assert.assertEquals(2, taskHistory.size()); + + runningTasks.get(0).setState(TaskState.SUCCESS); + succeededTasks++; + while (getNumSubTasks("numSucceededTasks", req -> task.getNumSucceededTasks(req)) < succeededTasks) { + Thread.sleep(100); + } + + Assert.assertEquals(TaskState.SUCCESS, supervisorTaskFuture.get(100, TimeUnit.MILLISECONDS).getStatusCode()); + } + + private int getNumSubTasks(String name, Function func) + { + final Response response = func.apply(newRequest()); + Assert.assertEquals(200, response.getStatus()); + final Map numTasks = (Map) response.getEntity(); + return numTasks.get(name); + } + + private Map buildStateMap() + { + final Map stateMap = new HashMap<>(); + subTaskSpecs.forEach((specId, spec) -> { + final List taskHistory = taskHistories.get(specId); + final TaskStatusPlus runningTaskStatus = runningSpecs.get(specId); + stateMap.put( + specId, + new SubTaskStateResponse(spec, runningTaskStatus, taskHistory == null ? Collections.emptyList() : taskHistory) + ); + }); + return stateMap; + } + + /** + * Test all endpoints of {@link SinglePhaseParallelIndexSupervisorTask}. + */ + private void checkState( + TestSupervisorTask task, + int expectedSucceededTasks, + int expectedFailedTask, + Map expectedSubTaskStateResponses // subTaskSpecId -> response + ) + { + // numRunningTasks + Response response = task.getNumRunningTasks(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of("numRunningTasks", runningTasks.size()), + response.getEntity() + ); + + // numSucceededTasks + response = task.getNumSucceededTasks(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("numSucceededTasks", expectedSucceededTasks), response.getEntity()); + + // numFailedTasks + response = task.getNumFailedTasks(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("numFailedTasks", expectedFailedTask), response.getEntity()); + + // numCompleteTasks + response = task.getNumCompleteTasks(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of("numCompleteTasks", expectedSucceededTasks + expectedFailedTask), + response.getEntity() + ); + + // runningSubTasks + response = task.getRunningTasks(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of("runningSubTasks", runningTasks.stream().map(AbstractTask::getId).collect(Collectors.toSet())), + response.getEntity() + ); + + // subTaskSpecs + response = task.getSubTaskSpecs(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Map>> actualSubTaskSpecMap = + (Map>>) response.getEntity(); + Assert.assertEquals( + subTaskSpecs.keySet(), + actualSubTaskSpecMap.get("subTaskSpecs").stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) + ); + + // runningSubTaskSpecs + response = task.getRunningSubTaskSpecs(newRequest()); + Assert.assertEquals(200, response.getStatus()); + actualSubTaskSpecMap = + (Map>>) response.getEntity(); + Assert.assertEquals( + runningSpecs.keySet(), + actualSubTaskSpecMap.get("runningSubTaskSpecs").stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) + ); + + // completeSubTaskSpecs + final List> completeSubTaskSpecs = expectedSubTaskStateResponses + .entrySet() + .stream() + .filter(entry -> !runningSpecs.containsKey(entry.getKey())) + .map(entry -> entry.getValue().getSpec()) + .collect(Collectors.toList()); + + response = task.getCompleteSubTaskSpecs(newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("completeSubTaskSpecs", completeSubTaskSpecs), response.getEntity()); + + // subTaskSpec + final String subTaskId = runningSpecs.keySet().iterator().next(); + response = task.getSubTaskSpec(subTaskId, newRequest()); + Assert.assertEquals(200, response.getStatus()); + final SubTaskSpec subTaskSpec = + (SubTaskSpec) response.getEntity(); + Assert.assertEquals(subTaskId, subTaskSpec.getId()); + + // subTaskState + response = task.getSubTaskState(subTaskId, newRequest()); + Assert.assertEquals(200, response.getStatus()); + final SubTaskStateResponse expectedResponse = Preconditions.checkNotNull( + expectedSubTaskStateResponses.get(subTaskId), + "response for task[%s]", + subTaskId + ); + final SubTaskStateResponse actualResponse = (SubTaskStateResponse) response.getEntity(); + Assert.assertEquals(expectedResponse.getSpec().getId(), actualResponse.getSpec().getId()); + Assert.assertEquals(expectedResponse.getCurrentStatus(), actualResponse.getCurrentStatus()); + Assert.assertEquals(expectedResponse.getTaskHistory(), actualResponse.getTaskHistory()); + + // completeSubTaskSpecAttemptHistory + final String completeSubTaskSpecId = expectedSubTaskStateResponses + .entrySet() + .stream() + .filter(entry -> { + final TaskStatusPlus currentStatus = entry.getValue().getCurrentStatus(); + return currentStatus != null && + (currentStatus.getState() == TaskState.SUCCESS || currentStatus.getState() == TaskState.FAILED); + }) + .map(Entry::getKey) + .findFirst() + .orElse(null); + if (completeSubTaskSpecId != null) { + response = task.getCompleteSubTaskSpecAttemptHistory(completeSubTaskSpecId, newRequest()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + expectedSubTaskStateResponses.get(completeSubTaskSpecId).getTaskHistory(), + response.getEntity() + ); + } + } + + private static HttpServletRequest newRequest() + { + final HttpServletRequest request = EasyMock.niceMock(HttpServletRequest.class); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(new AuthenticationResult("test", "test", Collections.emptyMap())); + EasyMock.replay(request); + return request; + } + + private TestSupervisorTask newTask( + Interval interval, + SinglePhaseParallelIndexIOConfig ioConfig + ) + { + // set up ingestion spec + final SinglePhaseParallelIndexIngestionSpec singlePhaseIngestionSpec = new SinglePhaseParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + getObjectMapper().convertValue( + new StringInputRowParser( + DEFAULT_PARSE_SPEC, + null + ), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null, + getObjectMapper() + ), + ioConfig, + new SinglePhaseParallelIndexTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + NUM_SUB_TASKS, + null, + null, + null, + null, + null, + null, + null + ) + ); + + // set up test tools + return new TestSupervisorTask( + null, + null, + singlePhaseIngestionSpec, + new HashMap<>(), + indexingServiceClient + ); + } + + private static class TestFirehose implements FiniteFirehoseFactory + { + private final List ids; + + TestFirehose(List ids) + { + this.ids = ids; + } + + @Override + public Stream> getSplits() + { + return ids.stream().map(InputSplit::new); + } + + @Override + public int getNumSplits() + { + return ids.size(); + } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + return new TestFirehose(Collections.singletonList(split.get())); + } + } + + private class TestSupervisorTask extends TestSinglePhaseParallelIndexSupervisorTask + { + TestSupervisorTask( + String id, + TaskResource taskResource, + SinglePhaseParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + super( + id, + taskResource, + ingestionSchema, + context, + indexingServiceClient + ); + } + + @Override + SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + { + final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() + .getIOConfig() + .getFirehoseFactory(); + final TestSubTaskSpec spec = new TestSubTaskSpec( + getId() + "_" + getAndIncrementNextSpecId(), + getGroupId(), + this, + new SinglePhaseParallelIndexIngestionSpec( + getIngestionSchema().getDataSchema(), + new SinglePhaseParallelIndexIOConfig( + baseFirehoseFactory.withSplit(split), + getIngestionSchema().getIOConfig().isAppendToExisting() + ), + getIngestionSchema().getTuningConfig() + ), + getContext(), + split + ); + subTaskSpecs.put(spec.getId(), spec); + return spec; + } + } + + private class TestSubTaskSpec extends SinglePhaseParallelIndexSubTaskSpec + { + private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + + TestSubTaskSpec( + String id, + String groupId, + SinglePhaseParallelIndexSupervisorTask supervisorTask, + SinglePhaseParallelIndexIngestionSpec ingestionSpec, + Map context, + InputSplit inputSplit + ) + { + super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit); + this.supervisorTask = supervisorTask; + } + + @Override + public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) + { + try { + // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id + Thread.sleep(10); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + final TestSubTask subTask = new TestSubTask( + getGroupId(), + getSupervisorTaskId(), + numAttempts, + getIngestionSpec(), + getContext(), + new LocalSinglePhaseParallelIndexTaskClientFactory(supervisorTask) + ); + final TestFirehose firehose = (TestFirehose) getIngestionSpec().getIOConfig().getFirehoseFactory(); + final InputSplit split = firehose.getSplits().findFirst().orElse(null); + if (split == null) { + throw new ISE("Split is null"); + } + runningTasks.add(subTask); + taskIdToSpec.put(subTask.getId(), this); + runningSpecs.put( + getId(), + new TaskStatusPlus( + subTask.getId(), + subTask.getType(), + DateTimes.EPOCH, + DateTimes.EPOCH, + TaskState.RUNNING, + -1L, + TaskLocation.unknown(), + null, + null + ) + ); + return subTask; + } + } + + private class TestSubTask extends SinglePhaseParallelIndexSubTask + { + private volatile TaskState state = TaskState.RUNNING; + + TestSubTask( + String groupId, + String supervisorTaskId, + int numAttempts, + SinglePhaseParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexTaskClientFactory taskClientFactory + ) + { + super( + null, + groupId, + null, + supervisorTaskId, + numAttempts, + ingestionSchema, + context, + null, + taskClientFactory + ); + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) + { + return true; + } + + @Override + public TaskStatus run(final TaskToolbox toolbox) throws Exception + { + while (state == TaskState.RUNNING) { + Thread.sleep(100); + } + + final TestFirehose firehose = (TestFirehose) getIngestionSchema().getIOConfig().getFirehoseFactory(); + + final List locks = toolbox.getTaskActionClient() + .submit(new SurrogateLockListAction(getSupervisorTaskId())); + Preconditions.checkState(locks.size() == 1, "There should be a single lock"); + + task.collectReport( + new PushedSegmentsReport( + getId(), + Collections.singletonList( + new DataSegment( + getDataSource(), + Intervals.of("2017/2018"), + locks.get(0).getVersion(), + null, + null, + null, + new NumberedShardSpec(firehose.ids.get(0), NUM_SUB_TASKS), + 0, + 1L + ) + ) + ) + ); + return TaskStatus.fromCode(getId(), state); + } + + void setState(TaskState state) + { + Preconditions.checkArgument( + state == TaskState.SUCCESS || state == TaskState.FAILED, + "state[%s] should be SUCCESS of FAILED", + state + ); + this.state = state; + final int taskIndex = IntStream.range(0, runningTasks.size()) + .filter(i -> runningTasks.get(i).getId().equals(getId())).findAny() + .orElse(-1); + if (taskIndex == -1) { + throw new ISE("Can't find an index for task[%s]", getId()); + } + runningTasks.remove(taskIndex); + final String specId = Preconditions.checkNotNull(taskIdToSpec.get(getId()), "spec for task[%s]", getId()).getId(); + runningSpecs.remove(specId); + taskHistories.computeIfAbsent(specId, k -> new ArrayList<>()).add( + new TaskStatusPlus( + getId(), + getType(), + DateTimes.EPOCH, + DateTimes.EPOCH, + state, + -1L, + TaskLocation.unknown(), + null, + null + ) + ); + } + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java index 4bcdad224bf2..3110648a8638 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java @@ -19,97 +19,38 @@ package io.druid.indexing.common.task; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.indexing.IndexingServiceClient; -import io.druid.client.indexing.NoopIndexingServiceClient; -import io.druid.client.indexing.TaskStatusResponse; import io.druid.data.input.FiniteFirehoseFactory; import io.druid.data.input.InputSplit; -import io.druid.data.input.impl.CSVParseSpec; -import io.druid.data.input.impl.DimensionsSpec; -import io.druid.data.input.impl.ParseSpec; import io.druid.data.input.impl.StringInputRowParser; -import io.druid.data.input.impl.TimestampSpec; -import io.druid.indexer.TaskLocation; import io.druid.indexer.TaskState; -import io.druid.indexer.TaskStatusPlus; -import io.druid.indexing.common.TaskInfoProvider; -import io.druid.indexing.common.TaskStatus; -import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; -import io.druid.segment.loading.LocalDataSegmentPusher; -import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; -import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import io.druid.server.security.AuthorizerMapper; -import io.druid.timeline.DataSegment; -import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.stream.Stream; -public class SinglePhaseParallelIndexSupervisorTaskTest extends IngestionTestBase +public class SinglePhaseParallelIndexSupervisorTaskTest extends AbstractSinglePhaseParallelIndexSupervisorTaskTest { - @Rule - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")), - Lists.newArrayList(), - Lists.newArrayList() - ), - null, - Arrays.asList("ts", "dim", "val"), - false, - 0 - ); - - private TaskActionClient actionClient; - private LocalIndexingServiceClient indexingServiceClient; - private TaskToolbox toolbox; - private File localDeepStorage; private File inputDir; @Before @@ -291,7 +232,7 @@ private SinglePhaseParallelIndexSupervisorTask newTask( ); // set up test tools - return new TestSinglePhaseParallelIndexSupervisorTask( + return new TestSupervisorTask( null, null, singlePhaseIngestionSpec, @@ -300,9 +241,9 @@ private SinglePhaseParallelIndexSupervisorTask newTask( ); } - private static class TestSinglePhaseParallelIndexSupervisorTask extends SinglePhaseParallelIndexSupervisorTask + private static class TestSupervisorTask extends TestSinglePhaseParallelIndexSupervisorTask { - TestSinglePhaseParallelIndexSupervisorTask( + TestSupervisorTask( String id, TaskResource taskResource, SinglePhaseParallelIndexIngestionSpec ingestionSchema, @@ -315,30 +256,10 @@ private static class TestSinglePhaseParallelIndexSupervisorTask extends SinglePh taskResource, ingestionSchema, context, - indexingServiceClient, - new NoopChatHandlerProvider(), - new AuthorizerMapper(ImmutableMap.of()) + indexingServiceClient ); } - @Override - Stream subTaskSpecIterator() throws IOException - { - final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() - .getIOConfig() - .getFirehoseFactory(); - return baseFirehoseFactory.getSplits().map(split -> { - try { - // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id - Thread.sleep(10); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - return newTaskSpec((InputSplit) split); - }); - } - @Override SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) { @@ -346,7 +267,7 @@ SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) .getIOConfig() .getFirehoseFactory(); return new TestSinglePhaseParallelIndexSubTaskSpec( - getId() + "_" + getAndIncreaseNextSpecId(), + getId() + "_" + getAndIncrementNextSpecId(), getGroupId(), this, new SinglePhaseParallelIndexIngestionSpec( @@ -357,161 +278,12 @@ SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) ), getIngestionSchema().getTuningConfig() ), - getContext() + getContext(), + split ); } } - private class LocalIndexingServiceClient extends NoopIndexingServiceClient - { - private final ConcurrentMap> tasks = new ConcurrentHashMap<>(); - private final ListeningExecutorService service = MoreExecutors.listeningDecorator( - Execs.multiThreaded(5, "parallel-index-single-phase-supervisor-task-test-%d") - ); - - @Override - public String runTask(Object taskObject) - { - final SinglePhaseParallelIndexSubTask subTask = (SinglePhaseParallelIndexSubTask) taskObject; - tasks.put(subTask.getId(), service.submit(() -> { - try { - final TaskToolbox toolbox = createTaskToolbox(subTask); - if (subTask.isReady(toolbox.getTaskActionClient())) { - return subTask.run(toolbox); - } else { - throw new ISE("task[%s] is not ready", subTask.getId()); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - })); - return subTask.getId(); - } - - @Override - @Nullable - public TaskStatusResponse getTaskStatus(String taskId) - { - final Future taskStatusFuture = tasks.get(taskId); - if (taskStatusFuture != null) { - try { - if (taskStatusFuture.isDone()) { - final TaskStatus taskStatus = taskStatusFuture.get(); - return new TaskStatusResponse( - taskId, - new TaskStatusPlus( - taskId, - "index_single_phase_sub", - DateTimes.EPOCH, - DateTimes.EPOCH, - taskStatus.getStatusCode(), - taskStatus.getDuration(), - TaskLocation.unknown(), - null, - null - ) - ); - } else { - return new TaskStatusResponse( - taskId, - new TaskStatusPlus( - taskId, - "index_single_phase_sub", - DateTimes.EPOCH, - DateTimes.EPOCH, - TaskState.RUNNING, - -1L, - TaskLocation.unknown(), - null, - null - ) - ); - } - } - catch (InterruptedException | ExecutionException e) { - // We don't have a way to pass this exception to the supervisorTask yet.. - // So, let's print it here. - System.err.println(Throwables.getStackTraceAsString(e)); - return new TaskStatusResponse( - taskId, - new TaskStatusPlus( - taskId, - "index_single_phase_sub", - DateTimes.EPOCH, - DateTimes.EPOCH, - TaskState.FAILED, - -1L, - TaskLocation.unknown(), - null, - null - ) - ); - } - } else { - return null; - } - } - - @Override - public String killTask(String taskId) - { - final Future taskStatusFuture = tasks.remove(taskId); - if (taskStatusFuture != null) { - taskStatusFuture.cancel(true); - return taskId; - } else { - return null; - } - } - - void shutdown() - { - service.shutdownNow(); - } - } - - private TaskToolbox createTaskToolbox(Task task) throws IOException - { - return new TaskToolbox( - null, - actionClient, - null, - new LocalDataSegmentPusher( - new LocalDataSegmentPusherConfig() - { - @Override - public File getStorageDirectory() - { - return localDeepStorage; - } - }, - getObjectMapper() - ), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - getObjectMapper(), - temporaryFolder.newFolder(task.getId()), - getIndexIO(), - null, - null, - getIndexMerger(), - null, - null, - null, - null, - new NoopTestTaskFileWriter() - ); - } - private static class TestSinglePhaseParallelIndexSubTaskSpec extends SinglePhaseParallelIndexSubTaskSpec { private final SinglePhaseParallelIndexSupervisorTask supervisorTask; @@ -521,10 +293,11 @@ private static class TestSinglePhaseParallelIndexSubTaskSpec extends SinglePhase String groupId, SinglePhaseParallelIndexSupervisorTask supervisorTask, SinglePhaseParallelIndexIngestionSpec ingestionSpec, - Map context + Map context, + InputSplit inputSplit ) { - super(id, groupId, supervisorTask.getId(), ingestionSpec, context); + super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit); this.supervisorTask = supervisorTask; } @@ -544,44 +317,4 @@ public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) ); } } - - private static class LocalSinglePhaseParallelIndexTaskClientFactory - implements IndexTaskClientFactory - { - private final SinglePhaseParallelIndexSupervisorTask supervisorTask; - - LocalSinglePhaseParallelIndexTaskClientFactory(SinglePhaseParallelIndexSupervisorTask supervisorTask) - { - this.supervisorTask = supervisorTask; - } - - @Override - public SinglePhaseParallelIndexTaskClient build( - TaskInfoProvider taskInfoProvider, - String callerId, - int numThreads, - Duration httpTimeout, - long numRetries - ) - { - return new LocalSinglePhaseParallelIndexTaskClient(callerId, supervisorTask); - } - } - - private static class LocalSinglePhaseParallelIndexTaskClient extends SinglePhaseParallelIndexTaskClient - { - private final SinglePhaseParallelIndexSupervisorTask supervisorTask; - - public LocalSinglePhaseParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexSupervisorTask supervisorTask) - { - super(null, null, null, null, callerId, 0); - this.supervisorTask = supervisorTask; - } - - @Override - public void report(String supervisorTaskId, List pushedSegments) - { - supervisorTask.collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); - } - } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java index 96f1c34a420f..6913303567ad 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java @@ -22,12 +22,14 @@ import com.google.common.util.concurrent.ListenableFuture; import io.druid.client.indexing.NoopIndexingServiceClient; import io.druid.client.indexing.TaskStatusResponse; +import io.druid.data.input.InputSplit; import io.druid.indexer.TaskLocation; import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; +import io.druid.indexing.common.task.TaskMonitor.TaskHistory; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.concurrent.Execs; import org.junit.After; @@ -48,9 +50,11 @@ public class TaskMonitorTest { + private static final int SPLIT_NUM = 10; + private final ExecutorService taskRunner = Execs.multiThreaded(5, "task-monitor-test-%d"); private final ConcurrentMap tasks = new ConcurrentHashMap<>(); - private final TaskMonitor monitor = new TaskMonitor<>(new TestIndexingServiceClient(), 3); + private final TaskMonitor monitor = new TaskMonitor<>(new TestIndexingServiceClient(), 3, SPLIT_NUM); @Before public void setup() @@ -71,7 +75,9 @@ public void testBasic() throws InterruptedException, ExecutionException, Timeout { final List>> futures = IntStream .range(0, 10) - .mapToObj(i -> monitor.submit(new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, 100L, 0))) + .mapToObj(i -> monitor.submit( + new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, new IntegerInputSplit(i), 100L, 0) + )) .collect(Collectors.toList()); for (int i = 0; i < futures.size(); i++) { // # of threads of taskRunner is 5, so the expected max timeout is 2 sec. We additionally wait three more seconds @@ -88,9 +94,15 @@ public void testBasic() throws InterruptedException, ExecutionException, Timeout @Test public void testRetry() throws InterruptedException, ExecutionException, TimeoutException { - final List>> futures = IntStream + final List specs = IntStream .range(0, 10) - .mapToObj(i -> monitor.submit(new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, 100L, 2))) + .mapToObj( + i -> new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, new IntegerInputSplit(i), 100L, 2) + ) + .collect(Collectors.toList()); + final List>> futures = specs + .stream() + .map(monitor::submit) .collect(Collectors.toList()); for (int i = 0; i < futures.size(); i++) { // # of threads of taskRunner is 5, and each task is expected to be run 3 times (with 2 retries), so the expected @@ -103,7 +115,10 @@ public void testRetry() throws InterruptedException, ExecutionException, Timeout Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getState()); Assert.assertEquals(TaskState.SUCCESS, result.getLastState()); - final List attemptHistory = result.getAttemptHistory(); + final TaskHistory taskHistory = monitor.getCompleteSubTaskSpecHistory(specs.get(i).getId()); + Assert.assertNotNull(taskHistory); + + final List attemptHistory = taskHistory.getAttemptHistory(); Assert.assertNotNull(attemptHistory); Assert.assertEquals(3, attemptHistory.size()); Assert.assertEquals(TaskState.FAILED, attemptHistory.get(0).getState()); @@ -123,11 +138,12 @@ public TestTaskSpec( String groupId, String supervisorTaskId, Map context, + InputSplit inputSplit, long runTime, int numMaxFails ) { - super(id, groupId, supervisorTaskId, context); + super(id, groupId, supervisorTaskId, context, inputSplit); this.runTime = runTime; this.numMaxFails = numMaxFails; } @@ -191,4 +207,12 @@ public TaskStatusResponse getTaskStatus(String taskId) ); } } + + private static class IntegerInputSplit extends InputSplit + { + public IntegerInputSplit(int split) + { + super(split); + } + } } diff --git a/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java index 80ecee751b6c..ab52eb16c3c5 100644 --- a/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java @@ -246,7 +246,6 @@ private List getTasks(String endpointSuffix) } @Override - @Nullable public TaskStatusResponse getTaskStatus(String taskId) { try { @@ -254,16 +253,12 @@ public TaskStatusResponse getTaskStatus(String taskId) druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/task/%s/status", taskId)) ); - if (responseHolder.getStatus().equals(HttpResponseStatus.OK)) { - return jsonMapper.readValue( - responseHolder.getContent(), - new TypeReference() - { - } - ); - } else { - return null; - } + return jsonMapper.readValue( + responseHolder.getContent(), + new TypeReference() + { + } + ); } catch (IOException | InterruptedException e) { throw new RuntimeException(e); diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index c03997be82a3..5eac6aadcd1c 100644 --- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -57,7 +57,6 @@ String compactSegments( List getWaitingTasks(); - @Nullable TaskStatusResponse getTaskStatus(String taskId); @Nullable diff --git a/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java index 9d84f44fca12..7f7ba6e9b057 100644 --- a/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java +++ b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexer.TaskStatusPlus; +import javax.annotation.Nullable; import java.util.Objects; /** @@ -31,12 +32,13 @@ public class TaskStatusResponse { private final String task; // Task ID, named "task" in the JSONification of this class. + @Nullable private final TaskStatusPlus status; @JsonCreator public TaskStatusResponse( @JsonProperty("task") final String task, - @JsonProperty("status") final TaskStatusPlus status + @JsonProperty("status") @Nullable final TaskStatusPlus status ) { this.task = task; @@ -50,6 +52,7 @@ public String getTask() } @JsonProperty + @Nullable public TaskStatusPlus getStatus() { return status; @@ -72,7 +75,6 @@ public boolean equals(final Object o) @Override public int hashCode() { - return Objects.hash(task, status); } diff --git a/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java b/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java index aeb6024f513f..0c80d2b6a8d1 100644 --- a/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java +++ b/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java @@ -101,11 +101,10 @@ public List getWaitingTasks() return null; } - @Nullable @Override public TaskStatusResponse getTaskStatus(String taskId) { - return null; + return new TaskStatusResponse(taskId, null); } @Nullable From bd5d5d5357ef8e3cc147abd5a7e1a027109269f7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Apr 2018 09:53:25 -0700 Subject: [PATCH 17/47] increase timeout in test --- .../SinglePhaseParallelIndexSupervisorTaskResourceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java index 9bf2370b2cb6..f7c65a41a56a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java @@ -238,7 +238,7 @@ public void testAPIs() throws Exception Thread.sleep(100); } - Assert.assertEquals(TaskState.SUCCESS, supervisorTaskFuture.get(100, TimeUnit.MILLISECONDS).getStatusCode()); + Assert.assertEquals(TaskState.SUCCESS, supervisorTaskFuture.get(1000, TimeUnit.MILLISECONDS).getStatusCode()); } private int getNumSubTasks(String name, Function func) From 9d5d7c02af3c3279979b5f9cea8a2db9ea7e2357 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Apr 2018 12:20:36 -0700 Subject: [PATCH 18/47] Added doc --- docs/content/ingestion/native_tasks.md | 232 +++++++++++++++++- .../indexing/common/IndexTaskClient.java | 2 +- ...inglePhaseParallelIndexSupervisorTask.java | 93 +++---- .../SinglePhaseParallelIndexTaskClient.java | 2 +- ...rallelIndexSupervisorTaskResourceTest.java | 55 ++--- 5 files changed, 307 insertions(+), 77 deletions(-) diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index a06bf4025456..96e246804045 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -10,7 +10,7 @@ Single Phase Parallel Index Task The Single Phase Parallel Index Task is a task for parallel batch indexing. This task only uses Druid's resource and doesn't depend on other external systems like Hadoop. This task works in a single phase without shuffling intermediate -data. `index_single_phase_parallel` task is a supervisor task which basically creates multiple worker tasks and submits +data. `index_single_phase_parallel` task is a supervisor task which basically generates multiple worker tasks and submits them to overlords. Each worker task reads input data and makes segments. Once they successfully generate segments for all input, they report the generated segment list to the supervisor task. The supervisor task periodically checks the worker task statuses. If one of them fails, it retries the failed task until the retrying number reaches the configured limit. @@ -21,8 +21,10 @@ current splittable fireshoses are [`LocalFirehose`](./firehose.html#localfirehos , [`StaticS3Firehose`](../development/extensions-core/statics3firehose), [`StaticAzureBlobStoreFirehose`](../development/extensions-contrib/staticazureblobstorefirehose) , [`StaticGoogleBlobStoreFirehose`](../development/extensions-contrib/staticgoogleblobstorefirehose), and [`StaticCloudFilesFirehose`](../development/extensions-contrib/staticcloudfilesfirehose). -The splittable firehose is responsible for generating _splits_ which is an input of each worker tasks generated by the -supervisor task. As a result, the number of worker tasks depends on the implementation of splittable firehoses. +The splittable firehose is responsible for generating _splits_. The supervisor task generates _worker task specs_ each of +which specifies a split and submits worker tasks using those specs. As a result, the number of worker tasks depends on +the implementation of splittable firehoses. Please note that multiple tasks can be created for the same worker task spec +if one of them fails. Since this task doesn't shuffle intermediate data, it isn't available for [perfect rollup](../design/index.html). @@ -143,6 +145,230 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |chatHandlerTimeout|Timeout for reporting the pushed segments in worker tasks.|PT10S|no| |chatHandlerNumRetries|Retries for reporting the pushed segments in worker tasks.|5|no| +#### HTTP Endpoints + +The supervisor task provides some HTTP endpoints to get running status. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/isRunningInParallel` + +Returns the indexing task is running in parallel. It returns false if the firehose in IOConfig is not splittable. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numRunningTasks` + +Returns the number of running worker tasks or 0 if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numSucceededTasks` + +Returns the number of succeeded worker tasks or 0 if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numFailedTasks` + +Returns the number of failed worker tasks or 0 if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numCompleteTasks` + +Returns the number of succeeded or failed worker tasks, or 0 if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/expectedNumSucceededTasks` + +Returns the expected number of succeeded worker tasks to complete, or 0 if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/runningSubTasks` + +Returns the task IDs of running worker tasks, or an empty list if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subTaskSpecs` + +Returns all worker task specs, or an empty list if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/runningSubTaskSpecs` + +Returns running worker task specs, or an empty list if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/completeSubTaskSpecs` + +Returns complete worker task specs, or an empty list if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subTaskSpec?id={SUB_TASK_SPEC_ID}` + +Returns the worker task spec of the given id, or HTTP 404 Not Found error if isRunningInParallel is false. + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subTaskState?id={SUB_TASK_SPEC_ID}` + +Returns the state of the worker task spec of the given id, or HTTP 404 Not Found error if isRunningInParallel is false. +The returned result contains the worker task spec, a current task status if exists, and task attempt history. + +An example of the result is + +```json +{ + "spec": { + "id": "index_single_phase_parallel_lineitem_2018-04-20T22:12:43.610Z_2", + "groupId": "index_single_phase_parallel_lineitem_2018-04-20T22:12:43.610Z", + "supervisorTaskId": "index_single_phase_parallel_lineitem_2018-04-20T22:12:43.610Z", + "context": null, + "inputSplit": { + "split": "/path/to/data/lineitem.tbl.5" + }, + "ingestionSpec": { + "dataSchema": { + "dataSource": "lineitem", + "parser": { + "type": "hadoopyString", + "parseSpec": { + "format": "tsv", + "delimiter": "|", + "timestampSpec": { + "column": "l_shipdate", + "format": "yyyy-MM-dd" + }, + "dimensionsSpec": { + "dimensions": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ] + }, + "columns": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ] + } + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "longSum", + "name": "l_quantity", + "fieldName": "l_quantity", + "expression": null + }, + { + "type": "doubleSum", + "name": "l_extendedprice", + "fieldName": "l_extendedprice", + "expression": null + }, + { + "type": "doubleSum", + "name": "l_discount", + "fieldName": "l_discount", + "expression": null + }, + { + "type": "doubleSum", + "name": "l_tax", + "fieldName": "l_tax", + "expression": null + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "YEAR", + "queryGranularity": { + "type": "none" + }, + "rollup": true, + "intervals": [ + "1980-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z" + ] + }, + "transformSpec": { + "filter": null, + "transforms": [] + } + }, + "ioConfig": { + "type": "index_single_phase_parallel", + "firehose": { + "type": "local", + "baseDir": "/path/to/data/", + "filter": "lineitem.tbl.5", + "parser": null + }, + "appendToExisting": false + }, + "tuningConfig": { + "type": "index_single_phase_parallel", + "targetPartitionSize": 5000000, + "maxRowsInMemory": 500000, + "maxTotalRows": 20000000, + "numShards": null, + "indexSpec": { + "bitmap": { + "type": "concise" + }, + "dimensionCompression": "lz4", + "metricCompression": "lz4", + "longEncoding": "longs" + }, + "maxPendingPersists": 0, + "forceExtendableShardSpecs": false, + "reportParseExceptions": false, + "pushTimeout": 0, + "segmentWriteOutMediumFactory": null, + "maxNumBatchTasks": 2147483647, + "maxRetry": 3, + "taskStatusCheckPeriodMs": 1000, + "chatHandlerTimeout": "PT10S", + "chatHandlerNumRetries": 5, + "logParseExceptions": false, + "maxParseExceptions": 2147483647, + "maxSavedParseExceptions": 0, + "forceGuaranteedRollup": false, + "buildV9Directly": true + } + } + }, + "currentStatus": { + "id": "index_single_phase_sub_lineitem_2018-04-20T22:16:29.922Z", + "type": "index_single_phase_sub", + "createdTime": "2018-04-20T22:16:29.925Z", + "queueInsertionTime": "2018-04-20T22:16:29.929Z", + "statusCode": "RUNNING", + "duration": -1, + "location": { + "host": null, + "port": -1, + "tlsPort": -1 + }, + "dataSource": "lineitem", + "errorMsg": null + }, + "taskHistory": [] +} +``` + +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/completeSubTaskSpecAttemptHistory?id={SUB_TASK_SPEC_ID}` + +Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if isRunningInParallel is false. + Local Index Task ---------------- diff --git a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java index 75aff7aa2488..40893cd995ec 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java @@ -200,7 +200,7 @@ protected FullResponseHolder submitJsonRequest( /** * To use this method, {@link #objectMapper} should be a smileMapper. */ - protected FullResponseHolder submitSmilRequest( + protected FullResponseHolder submitSmileRequest( String taskId, HttpMethod method, String pathSuffix, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java index 111003903a0a..f2508d979c7a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java @@ -26,7 +26,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -517,7 +516,7 @@ private static List getSplitsIfSplittable(FirehoseFactory firehoseFa public Response isRunningInParallel(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - return okResponse("isRunningInParallel", baseFirehoseFactory.isSplittable()); + return Response.ok(baseFirehoseFactory.isSplittable()).build(); } @GET @@ -527,7 +526,7 @@ public Response getNumRunningTasks(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); final int numRunningTasks = taskMonitor == null ? 0 : taskMonitor.getNumRunningTasks(); - return okResponse("numRunningTasks", numRunningTasks); + return Response.ok(numRunningTasks).build(); } @GET @@ -537,7 +536,7 @@ public Response getNumSucceededTasks(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); final int numSucceededTasks = taskMonitor == null ? 0 : taskMonitor.getNumSucceededTasks(); - return okResponse("numSucceededTasks", numSucceededTasks); + return Response.ok(numSucceededTasks).build(); } @GET @@ -547,7 +546,7 @@ public Response getNumFailedTasks(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); final int numFailedTasks = taskMonitor == null ? 0 : taskMonitor.getNumFailedTasks(); - return okResponse("numFailedTasks", numFailedTasks); + return Response.ok(numFailedTasks).build(); } @GET @@ -557,7 +556,7 @@ public Response getNumCompleteTasks(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); final int numCompleteTasks = taskMonitor == null ? 0 : taskMonitor.getNumCompleteTasks(); - return okResponse("numCompleteTasks", numCompleteTasks); + return Response.ok(numCompleteTasks).build(); } @GET @@ -567,7 +566,7 @@ public Response getExpectedNumSucceededTasks(@Context final HttpServletRequest r { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); final int expectedNumSucceededTasks = taskMonitor == null ? 0 : taskMonitor.getExpectedNumSucceededTasks(); - return okResponse("expectedNumSucceededTasks", expectedNumSucceededTasks); + return Response.ok(expectedNumSucceededTasks).build(); } @GET @@ -577,7 +576,7 @@ public Response getRunningTasks(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); final Set runningTasks = taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds(); - return okResponse("runningSubTasks", runningTasks); + return Response.ok(runningTasks).build(); } @GET @@ -586,17 +585,21 @@ public Response getRunningTasks(@Context final HttpServletRequest req) public Response getSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); - final List> completeSubTaskSpecs = taskMonitor - .getCompleteSubTaskSpecs(); - // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and - // completeSubTaskSpecs. - final Map> subTaskSpecMap = new HashMap<>( - runningSubTaskSpecs.size() + completeSubTaskSpecs.size() - ); - runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); - completeSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); - return okResponse("subTaskSpecs", new ArrayList<>(subTaskSpecMap.values())); + if (taskMonitor != null) { + final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); + final List> completeSubTaskSpecs = taskMonitor + .getCompleteSubTaskSpecs(); + // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and + // completeSubTaskSpecs. + final Map> subTaskSpecMap = new HashMap<>( + runningSubTaskSpecs.size() + completeSubTaskSpecs.size() + ); + runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); + completeSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); + return Response.ok(new ArrayList<>(subTaskSpecMap.values())).build(); + } else { + return Response.ok(Collections.emptyList()).build(); + } } @GET @@ -605,7 +608,10 @@ public Response getSubTaskSpecs(@Context final HttpServletRequest req) public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - return okResponse("runningSubTaskSpecs", taskMonitor.getRunningSubTaskSpecs()); + final List> runningSubTaskSpecs = taskMonitor == null ? + Collections.emptyList() : + taskMonitor.getRunningSubTaskSpecs(); + return Response.ok(runningSubTaskSpecs).build(); } @GET @@ -614,7 +620,11 @@ public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - return okResponse("completeSubTaskSpecs", taskMonitor.getCompleteSubTaskSpecs()); + final List> completeSubTaskSpecs = + taskMonitor == null ? + Collections.emptyList() : + taskMonitor.getCompleteSubTaskSpecs(); + return Response.ok(completeSubTaskSpecs).build(); } @GET @@ -623,26 +633,30 @@ public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) public Response getSubTaskSpec(@QueryParam("id") String id, @Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after - // adding them to taskHistory. - final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); - final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); - final SubTaskSpec subTaskSpec; - - if (monitorEntry != null) { - subTaskSpec = monitorEntry.getSpec(); - } else { - if (taskHistory != null) { - subTaskSpec = taskHistory.getSpec(); + if (taskMonitor != null) { + // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after + // adding them to taskHistory. + final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); + final SubTaskSpec subTaskSpec; + + if (monitorEntry != null) { + subTaskSpec = monitorEntry.getSpec(); } else { - subTaskSpec = null; + if (taskHistory != null) { + subTaskSpec = taskHistory.getSpec(); + } else { + subTaskSpec = null; + } } - } - if (subTaskSpec == null) { - return Response.status(Status.NOT_FOUND).build(); + if (subTaskSpec == null) { + return Response.status(Status.NOT_FOUND).build(); + } else { + return Response.ok(subTaskSpec).build(); + } } else { - return Response.ok(subTaskSpec).build(); + return Response.status(Status.NOT_FOUND).build(); } } @@ -710,11 +724,6 @@ public Response getCompleteSubTaskSpecAttemptHistory( } } - private static Response okResponse(String key, Object val) - { - return Response.ok(ImmutableMap.of(key, val)).build(); - } - static class SubTaskStateResponse { private final SinglePhaseParallelIndexSubTaskSpec spec; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java index 4084b634a5dd..fb8e109aaee3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java @@ -57,7 +57,7 @@ public String getSubtaskId() public void report(String supervisorTaskId, List pushedSegments) { try { - final FullResponseHolder response = submitSmilRequest( + final FullResponseHolder response = submitSmileRequest( supervisorTaskId, HttpMethod.POST, "report", diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java index f7c65a41a56a..645382ed3990 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java @@ -20,7 +20,6 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; import io.druid.client.indexing.IndexingServiceClient; import io.druid.data.input.FiniteFirehoseFactory; import io.druid.data.input.InputSplit; @@ -58,8 +57,10 @@ import javax.ws.rs.core.Response; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -133,16 +134,16 @@ public void testAPIs() throws Exception // test isRunningInParallel Response response = task.isRunningInParallel(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(ImmutableMap.of("isRunningInParallel", true), response.getEntity()); + Assert.assertTrue((Boolean) response.getEntity()); // test expectedNumSucceededTasks response = task.getExpectedNumSucceededTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(ImmutableMap.of("expectedNumSucceededTasks", NUM_SUB_TASKS), response.getEntity()); + Assert.assertEquals(NUM_SUB_TASKS, response.getEntity()); // Since taskMonitor works based on polling, it's hard to use a fancier way to check its state. // We use polling to check the state of taskMonitor in this test. - while (getNumSubTasks("numRunningTasks", req -> task.getNumRunningTasks(req)) < NUM_SUB_TASKS) { + while (getNumSubTasks(req -> task.getNumRunningTasks(req)) < NUM_SUB_TASKS) { Thread.sleep(100); } @@ -161,7 +162,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks("numSucceededTasks", req -> task.getNumSucceededTasks(req)) < succeededTasks) { + while (getNumSubTasks(req -> task.getNumSucceededTasks(req)) < succeededTasks) { Thread.sleep(100); } @@ -178,7 +179,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.FAILED); } - while (getNumSubTasks("numFailedTasks", req -> task.getNumFailedTasks(req)) < failedTasks) { + while (getNumSubTasks(req -> task.getNumFailedTasks(req)) < failedTasks) { Thread.sleep(100); } @@ -200,7 +201,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks("numSucceededTasks", req -> task.getNumSucceededTasks(req)) < succeededTasks) { + while (getNumSubTasks(req -> task.getNumSucceededTasks(req)) < succeededTasks) { Thread.sleep(100); } @@ -220,7 +221,7 @@ public void testAPIs() throws Exception // Test one more failure runningTasks.get(0).setState(TaskState.FAILED); failedTasks++; - while (getNumSubTasks("numFailedTasks", req -> task.getNumFailedTasks(req)) < failedTasks) { + while (getNumSubTasks(req -> task.getNumFailedTasks(req)) < failedTasks) { Thread.sleep(100); } @@ -234,19 +235,19 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); succeededTasks++; - while (getNumSubTasks("numSucceededTasks", req -> task.getNumSucceededTasks(req)) < succeededTasks) { + while (getNumSubTasks(req -> task.getNumSucceededTasks(req)) < succeededTasks) { Thread.sleep(100); } Assert.assertEquals(TaskState.SUCCESS, supervisorTaskFuture.get(1000, TimeUnit.MILLISECONDS).getStatusCode()); } - private int getNumSubTasks(String name, Function func) + @SuppressWarnings({"ConstantConditions"}) + private int getNumSubTasks(Function func) { final Response response = func.apply(newRequest()); Assert.assertEquals(200, response.getStatus()); - final Map numTasks = (Map) response.getEntity(); - return numTasks.get(name); + return (Integer) response.getEntity(); } private Map buildStateMap() @@ -276,55 +277,49 @@ private void checkState( // numRunningTasks Response response = task.getNumRunningTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals( - ImmutableMap.of("numRunningTasks", runningTasks.size()), - response.getEntity() - ); + Assert.assertEquals(runningTasks.size(), response.getEntity()); // numSucceededTasks response = task.getNumSucceededTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(ImmutableMap.of("numSucceededTasks", expectedSucceededTasks), response.getEntity()); + Assert.assertEquals(expectedSucceededTasks, response.getEntity()); // numFailedTasks response = task.getNumFailedTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(ImmutableMap.of("numFailedTasks", expectedFailedTask), response.getEntity()); + Assert.assertEquals(expectedFailedTask, response.getEntity()); // numCompleteTasks response = task.getNumCompleteTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals( - ImmutableMap.of("numCompleteTasks", expectedSucceededTasks + expectedFailedTask), - response.getEntity() - ); + Assert.assertEquals(expectedSucceededTasks + expectedFailedTask, response.getEntity()); // runningSubTasks response = task.getRunningTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals( - ImmutableMap.of("runningSubTasks", runningTasks.stream().map(AbstractTask::getId).collect(Collectors.toSet())), - response.getEntity() + runningTasks.stream().map(AbstractTask::getId).collect(Collectors.toSet()), + new HashSet<>((Collection) response.getEntity()) ); // subTaskSpecs response = task.getSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); - Map>> actualSubTaskSpecMap = - (Map>>) response.getEntity(); + List> actualSubTaskSpecMap = + (List>) response.getEntity(); Assert.assertEquals( subTaskSpecs.keySet(), - actualSubTaskSpecMap.get("subTaskSpecs").stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) + actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) ); // runningSubTaskSpecs response = task.getRunningSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); actualSubTaskSpecMap = - (Map>>) response.getEntity(); + (List>) response.getEntity(); Assert.assertEquals( runningSpecs.keySet(), - actualSubTaskSpecMap.get("runningSubTaskSpecs").stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) + actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) ); // completeSubTaskSpecs @@ -337,7 +332,7 @@ private void checkState( response = task.getCompleteSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(ImmutableMap.of("completeSubTaskSpecs", completeSubTaskSpecs), response.getEntity()); + Assert.assertEquals(completeSubTaskSpecs, response.getEntity()); // subTaskSpec final String subTaskId = runningSpecs.keySet().iterator().next(); From 702b2a58327ced080898e6100f97a8f82a7ddb9b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Apr 2018 13:28:22 -0700 Subject: [PATCH 19/47] Address comments --- .../java/io/druid/indexer/TaskStatusPlus.java | 9 +++++---- docs/content/ingestion/native_tasks.md | 2 +- docs/content/ingestion/tasks.md | 2 +- .../firehose/s3/StaticS3FirehoseFactory.java | 20 +++++++++---------- 4 files changed, 17 insertions(+), 16 deletions(-) diff --git a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java index 8601e62ce7a3..66e3a6099f93 100644 --- a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java +++ b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java @@ -167,14 +167,15 @@ public int hashCode() public String toString() { return "TaskStatusPlus{" + - "id=" + id + - ", type=" + type + + "id='" + id + '\'' + + ", type='" + type + '\'' + ", createdTime=" + createdTime + ", queueInsertionTime=" + queueInsertionTime + ", state=" + state + ", duration=" + duration + ", location=" + location + - ", dataSource=" + dataSource + - "}"; + ", dataSource='" + dataSource + '\'' + + ", errorMsg='" + errorMsg + '\'' + + '}'; } } diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 96e246804045..1def6e2a75a1 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -3,7 +3,7 @@ layout: doc_page --- # Native Index Tasks -Druid currently two types of native index tasks, i.e., `local index task` and `single phase parallel index task`. +Druid currently has two types of native index tasks, i.e., `local index task` and `single phase parallel index task`. Single Phase Parallel Index Task -------------------------------- diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index fc39c9ef4b7c..5ad8d06e4dd5 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -15,7 +15,7 @@ See [batch ingestion](../ingestion/batch-ingestion.html). ### Native Index Tasks -Druid provides a native index tasks which doesn't need any dependencies on other systems. +Druid provides a native index task which doesn't need any dependencies on other systems. See [native index tasks](./native_tasks.html) for more details. Segment Merging Tasks diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 8583f32bd3f1..58349a606df3 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -264,16 +264,16 @@ protected Predicate getRetryCondition() @Override public FiniteFirehoseFactory withSplit(InputSplit split) { - final String authority = split.get().getBucketName(); - final String path = split.get().getKey(); - final URI splitUri; - if (authority.endsWith("/") && path.startsWith("/")) { - splitUri = URI.create(StringUtils.format("s3://%s%s", authority, path.substring(1))); - } else if (!authority.endsWith("/") && !path.startsWith("/")) { - splitUri = URI.create(StringUtils.format("s3://%s/%s", authority, path)); - } else { - splitUri = URI.create(StringUtils.format("s3://%s%s", authority, path)); - } + final String originalAuthority = split.get().getBucketName(); + final String originalPath = split.get().getKey(); + final String authority = originalAuthority.endsWith("/") ? + originalAuthority.substring(0, originalAuthority.length() - 1) : + originalAuthority; + final String path = originalPath.startsWith("/") ? + originalPath.substring(1, originalPath.length()) : + originalPath; + + final URI splitUri = URI.create(StringUtils.format("s3://%s/%s", authority, path)); return new StaticS3FirehoseFactory( s3Client, Collections.singletonList(splitUri), From 6d32f0f7701f7a9aaa1f12a9af8386545947ec19 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Apr 2018 11:38:50 -0700 Subject: [PATCH 20/47] Fix overlapping locks --- .../druid/indexing/kafka/KafkaIndexTask.java | 15 +- .../ActionBasedSegmentAllocator.java | 16 +- .../CountingActionBasedSegmentAllocator.java | 6 - .../SegmentAllocateActionGenerator.java | 35 +++ .../CountingSegmentAllocateAction.java | 27 +- ...ckListAction.java => SurrogateAction.java} | 34 ++- .../SurrogateLockTryAcquireAction.java | 108 -------- .../indexing/common/actions/TaskAction.java | 8 +- .../AppenderatorDriverRealtimeIndexTask.java | 15 +- .../druid/indexing/common/task/IndexTask.java | 15 +- .../task/SinglePhaseParallelIndexSubTask.java | 242 ++++-------------- .../CountingSegmentAllocateActionTest.java | 6 - .../common/actions/SurrogateActionTest.java | 45 ++++ ...rallelIndexSupervisorTaskResourceTest.java | 5 +- ...ePhaseParallelIndexSupervisorTaskTest.java | 1 + 15 files changed, 209 insertions(+), 369 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java rename indexing-service/src/main/java/io/druid/indexing/common/actions/{SurrogateLockListAction.java => SurrogateAction.java} (69%) delete mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index ab69c43b212d..d735e92f5a2c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -60,6 +60,7 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.CheckPointDataSourceMetadataAction; import io.druid.indexing.common.actions.ResetDataSourceMetadataAction; +import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.AbstractTask; @@ -1934,7 +1935,19 @@ private StreamAppenderatorDriver newDriver( { return new StreamAppenderatorDriver( appenderator, - new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ), toolbox.getSegmentHandoffNotifierFactory(), new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java index 28a072d14cb5..cf59fc726f88 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/ActionBasedSegmentAllocator.java @@ -20,7 +20,6 @@ package io.druid.indexing.appenderator; import io.druid.data.input.InputRow; -import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.segment.indexing.DataSchema; import io.druid.segment.realtime.appenderator.SegmentAllocator; @@ -32,14 +31,17 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator { private final TaskActionClient taskActionClient; private final DataSchema dataSchema; + private final SegmentAllocateActionGenerator actionGenerator; public ActionBasedSegmentAllocator( TaskActionClient taskActionClient, - DataSchema dataSchema + DataSchema dataSchema, + SegmentAllocateActionGenerator actionGenerator ) { this.taskActionClient = taskActionClient; this.dataSchema = dataSchema; + this.actionGenerator = actionGenerator; } @Override @@ -51,15 +53,7 @@ public SegmentIdentifier allocate( ) throws IOException { return taskActionClient.submit( - new SegmentAllocateAction( - dataSchema.getDataSource(), - row.getTimestamp(), - dataSchema.getGranularitySpec().getQueryGranularity(), - dataSchema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) + actionGenerator.generate(dataSchema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java index 4319d2e0cc0f..b9956b64d0f2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java @@ -26,11 +26,9 @@ import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.realtime.appenderator.SegmentAllocator; import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import io.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; import java.io.IOException; -import java.util.List; import java.util.Map; public class CountingActionBasedSegmentAllocator implements SegmentAllocator @@ -38,21 +36,18 @@ public class CountingActionBasedSegmentAllocator implements SegmentAllocator private final TaskActionClient taskActionClient; private final String dataSource; private final GranularitySpec granularitySpec; - private final Map> shardSpecs; private final Map versions; public CountingActionBasedSegmentAllocator( TaskActionClient taskActionClient, String dataSource, GranularitySpec granularitySpec, - Map> shardSpecs, Map versions ) { this.taskActionClient = Preconditions.checkNotNull(taskActionClient, "taskActionClient"); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.granularitySpec = Preconditions.checkNotNull(granularitySpec, "granularitySpec"); - this.shardSpecs = Preconditions.checkNotNull(shardSpecs, "shardSpecs"); this.versions = Preconditions.checkNotNull(versions, "versions"); } @@ -69,7 +64,6 @@ public SegmentIdentifier allocate( dataSource, row.getTimestamp(), granularitySpec, - shardSpecs, versions ) ); diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java new file mode 100644 index 000000000000..768815fa6e06 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java @@ -0,0 +1,35 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.appenderator; + +import io.druid.data.input.InputRow; +import io.druid.indexing.common.actions.TaskAction; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; + +public interface SegmentAllocateActionGenerator +{ + TaskAction generate( + DataSchema dataSchema, + InputRow row, + String sequenceName, + String previousSegmentId, + boolean skipSegmentLineageCheck + ); +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java index 27cc538ca75b..88f634b6e0ba 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java @@ -34,14 +34,13 @@ import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.timeline.partition.NumberedShardSpec; -import io.druid.timeline.partition.ShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; +import java.util.SortedSet; public class CountingSegmentAllocateAction implements TaskAction { @@ -49,24 +48,24 @@ public class CountingSegmentAllocateAction implements TaskAction> shardSpecs; - @JsonDeserialize(keyUsing = IntervalDeserializer.class) private final Map versions; + private final SortedSet bucketIntervals; + @JsonCreator public CountingSegmentAllocateAction( @JsonProperty("dataSource") String dataSource, @JsonProperty("timestamp") DateTime timestamp, @JsonProperty("granularitySpec") GranularitySpec granularitySpec, - @JsonProperty("shardSpecs") Map> shardSpecs, @JsonProperty("versions") Map versions ) { this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.timestamp = Preconditions.checkNotNull(timestamp, "timestamp"); this.granularitySpec = Preconditions.checkNotNull(granularitySpec, "granularitySpec"); - this.shardSpecs = Preconditions.checkNotNull(shardSpecs, "shardSpecs"); this.versions = Preconditions.checkNotNull(versions, "versions"); + + this.bucketIntervals = Preconditions.checkNotNull(granularitySpec.bucketIntervals().orNull(), "bucketIntervals"); } @JsonProperty @@ -87,12 +86,6 @@ public GranularitySpec getGranularitySpec() return granularitySpec; } - @JsonProperty - public Map> getShardSpecs() - { - return shardSpecs; - } - @JsonProperty public Map getVersions() { @@ -116,8 +109,8 @@ public SegmentIdentifier perform(Task task, TaskActionToolbox toolbox) } final Interval interval = maybeInterval.get(); - if (!shardSpecs.containsKey(interval)) { - throw new ISE("Could not find shardSpec for interval[%s]", interval); + if (!bucketIntervals.contains(interval)) { + throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); } final Counters counters = toolbox.getCounters(); @@ -144,7 +137,6 @@ public String toString() "dataSource='" + dataSource + '\'' + ", timestamp=" + timestamp + ", granularitySpec=" + granularitySpec + - ", shardSpecs=" + shardSpecs + ", versions=" + versions + '}'; } @@ -170,16 +162,13 @@ public boolean equals(Object o) if (!granularitySpec.equals(that.granularitySpec)) { return false; } - if (!shardSpecs.equals(that.shardSpecs)) { - return false; - } return versions.equals(that.versions); } @Override public int hashCode() { - return Objects.hash(dataSource, timestamp, granularitySpec, shardSpecs, versions); + return Objects.hash(dataSource, timestamp, granularitySpec, versions); } private static String findVersion(Map versions, Interval interval) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockListAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java similarity index 69% rename from indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockListAction.java rename to indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java index 30280ded262c..01b98b2cc0b5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockListAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java @@ -16,29 +16,28 @@ * specific language governing permissions and limitations * under the License. */ - package io.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Optional; -import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.task.Task; import io.druid.java.util.common.ISE; -import java.util.List; - -public class SurrogateLockListAction implements TaskAction> +public class SurrogateAction> implements TaskAction { private final String surrogateId; + private final T taskAction; @JsonCreator - public SurrogateLockListAction( - @JsonProperty("surrogateId") String surrogateId + public SurrogateAction( + @JsonProperty("surrogateId") String surrogateId, + @JsonProperty("taskAction") T taskAction ) { this.surrogateId = surrogateId; + this.taskAction = taskAction; } @JsonProperty @@ -47,18 +46,24 @@ public String getSurrogateId() return surrogateId; } + @JsonProperty + public T getTaskAction() + { + return taskAction; + } + @Override - public TypeReference> getReturnTypeReference() + public TypeReference getReturnTypeReference() { - return new TypeReference>() {}; + return taskAction.getReturnTypeReference(); } @Override - public List perform(Task task, TaskActionToolbox toolbox) + public R perform(Task task, TaskActionToolbox toolbox) { final Optional maybeSurrogateTask = toolbox.getTaskStorage().getTask(surrogateId); if (maybeSurrogateTask.isPresent()) { - return toolbox.getTaskLockbox().findLocksForTask(maybeSurrogateTask.get()); + return taskAction.perform(maybeSurrogateTask.get(), toolbox); } else { throw new ISE("Can't find surrogate task[%s]", surrogateId); } @@ -67,12 +72,15 @@ public List perform(Task task, TaskActionToolbox toolbox) @Override public boolean isAudited() { - return false; + return taskAction.isAudited(); } @Override public String toString() { - return "LockListAction{}"; + return "SurrogateAction{" + + "surrogateId='" + surrogateId + '\'' + + ", taskAction=" + taskAction + + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java deleted file mode 100644 index 316466825d4c..000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateLockTryAcquireAction.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Optional; -import io.druid.indexing.common.TaskLock; -import io.druid.indexing.common.TaskLockType; -import io.druid.indexing.common.task.Task; -import io.druid.indexing.overlord.LockResult; -import io.druid.java.util.common.ISE; -import org.joda.time.Interval; - -public class SurrogateLockTryAcquireAction implements TaskAction -{ - private final TaskLockType type; - - private final Interval interval; - - private final String surrogateId; - - @JsonCreator - public SurrogateLockTryAcquireAction( - @JsonProperty("lockType") TaskLockType type, - @JsonProperty("interval") Interval interval, - @JsonProperty("surrogateId") String surrogateId - ) - { - this.type = type; - this.interval = interval; - this.surrogateId = surrogateId; - } - - @JsonProperty("lockType") - public TaskLockType getType() - { - return type; - } - - @JsonProperty("interval") - public Interval getInterval() - { - return interval; - } - - @JsonProperty("surrogateId") - public String getSurrogateId() - { - return surrogateId; - } - - @Override - public TypeReference getReturnTypeReference() - { - return new TypeReference() - { - }; - } - - @Override - public TaskLock perform( - Task task, TaskActionToolbox toolbox - ) - { - final Optional maybeSurrogateTask = toolbox.getTaskStorage().getTask(surrogateId); - if (maybeSurrogateTask.isPresent()) { - final LockResult result = toolbox.getTaskLockbox().tryLock(type, maybeSurrogateTask.get(), interval); - return result.isOk() ? result.getTaskLock() : null; - } else { - throw new ISE("Can't find surrogate task[%s]", surrogateId); - } - } - - @Override - public boolean isAudited() - { - return false; - } - - @Override - public String toString() - { - return "SurrogateLockTryAcquireAction{" + - "lockType=" + type + - ", interval=" + interval + - ", surrogateId=" + surrogateId + - '}'; - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index 368909b851dc..1b73a38b38f1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -28,9 +28,7 @@ @JsonSubTypes(value = { @JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class), @JsonSubTypes.Type(name = "lockTryAcquire", value = LockTryAcquireAction.class), - @JsonSubTypes.Type(name = "surrogateLockTryAcquire", value = SurrogateLockTryAcquireAction.class), @JsonSubTypes.Type(name = "lockList", value = LockListAction.class), - @JsonSubTypes.Type(name = "surrogateLockList", value = SurrogateLockListAction.class), @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @@ -41,11 +39,15 @@ @JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class), @JsonSubTypes.Type(name = "countingSegmentAllocate", value = CountingSegmentAllocateAction.class), @JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class), - @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class) + @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class), + @JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class) }) public interface TaskAction { TypeReference getReturnTypeReference(); // T_T RetType perform(Task task, TaskActionToolbox toolbox); boolean isAudited(); + + @Override + String toString(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 9e2751ffe7ff..d37aa19d0fba 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -47,6 +47,7 @@ import io.druid.indexing.common.TaskReport; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec; @@ -706,7 +707,19 @@ private static StreamAppenderatorDriver newDriver( { return new StreamAppenderatorDriver( appenderator, - new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema), + new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ), toolbox.getSegmentHandoffNotifierFactory(), new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), toolbox.getObjectMapper(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 80b049311e3e..8631ecf6a978 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -51,6 +51,7 @@ import io.druid.indexing.common.TaskReport; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; @@ -885,7 +886,19 @@ dataSchema, new RealtimeIOConfig(null, null, null), null segmentAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> lookup.get(sequenceName); } else if (ioConfig.isAppendToExisting()) { // Append mode: Allocate segments as needed using Overlord APIs. - segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); + segmentAllocator = new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ); } else { // Overwrite mode, non-guaranteed rollup: We can make up our own segment ids but we don't know them in advance. final Map counters = new HashMap<>(); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java index 83380951741b..e3828b26bb82 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.druid.client.indexing.IndexingServiceClient; @@ -38,18 +37,17 @@ import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.SurrogateLockListAction; -import io.druid.indexing.common.actions.SurrogateLockTryAcquireAction; +import io.druid.indexing.common.actions.LockListAction; +import io.druid.indexing.common.actions.LockTryAcquireAction; +import io.druid.indexing.common.actions.SegmentAllocateAction; +import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.IndexTask.IndexIOConfig; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; -import io.druid.indexing.common.task.IndexTask.ShardSpecs; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; import io.druid.query.DruidMetrics; @@ -67,9 +65,6 @@ import io.druid.segment.realtime.appenderator.SegmentAllocator; import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.HashBasedNumberedShardSpec; -import io.druid.timeline.partition.NumberedShardSpec; -import io.druid.timeline.partition.ShardSpec; import org.codehaus.plexus.util.FileUtils; import org.joda.time.Interval; @@ -80,12 +75,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.SortedSet; -import java.util.TreeSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -import java.util.function.Function; import java.util.stream.Collectors; /** @@ -192,10 +184,10 @@ public String getSupervisorTaskId() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { - final boolean determineIntervals = !ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals() - .isPresent(); + final boolean explicitIntervals = ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals() + .isPresent(); final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); @@ -208,37 +200,20 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Firehose temporary directory is automatically removed when this IndexTask completes. FileUtils.forceMkdir(firehoseTempDir); - final IndexTask.ShardSpecs shardSpecs = determineShardSpecs(firehoseFactory, firehoseTempDir); - - final DataSchema dataSchema; + final DataSchema dataSchema = ingestionSchema.getDataSchema(); final Map versions; - if (determineIntervals) { - final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - intervals.addAll(shardSpecs.getIntervals()); - final Map locks = tryAcquireExclusiveSurrogateLocks(toolbox.getTaskActionClient(), intervals); - versions = locks.entrySet().stream() - .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); - - dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( - ingestionSchema.getDataSchema() - .getGranularitySpec() - .withIntervals( - JodaUtils.condenseIntervals( - shardSpecs.getIntervals() - ) - ) - ); + + if (explicitIntervals) { + versions = toolbox.getTaskActionClient().submit(new SurrogateAction<>(supervisorTaskId, new LockListAction())) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); } else { - versions = toolbox.getTaskActionClient().submit(new SurrogateLockListAction(supervisorTaskId)) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - dataSchema = ingestionSchema.getDataSchema(); + versions = null; } final List pushedSegments = generateAndPushSegments( toolbox, dataSchema, - shardSpecs, versions, firehoseFactory, firehoseTempDir @@ -264,7 +239,9 @@ private Map tryAcquireExclusiveSurrogateLocks( final Map lockMap = new HashMap<>(); for (Interval interval : Tasks.computeCompactIntervals(intervals)) { final TaskLock lock = Preconditions.checkNotNull( - client.submit(new SurrogateLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval, supervisorTaskId)), + client.submit( + new SurrogateAction<>(supervisorTaskId, new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) + ), "Cannot acquire a lock for interval[%s]", interval ); lockMap.put(interval, lock); @@ -272,153 +249,6 @@ private Map tryAcquireExclusiveSurrogateLocks( return lockMap; } - /** - * Determines intervals and shardSpecs for input data. This method first checks that it must determine intervals and - * shardSpecs by itself. Intervals must be determined if they are not specified in {@link GranularitySpec}. - * ShardSpecs must be determined if the perfect rollup must be guaranteed even though the number of shards is not - * specified in {@link IndexTask.IndexTuningConfig}. - *

- * If both intervals and shardSpecs don't have to be determined, this method simply returns {@link IndexTask.ShardSpecs} for the - * given intervals. Here, if {@link IndexTask.IndexTuningConfig#numShards} is not specified, {@link NumberedShardSpec} is used. - *

- * If one of intervals or shardSpecs need to be determined, this method reads the entire input for determining one of - * them. If the perfect rollup must be guaranteed, {@link HashBasedNumberedShardSpec} is used for hash partitioning - * of input data. In the future we may want to also support single-dimension partitioning. - * - * @return generated {@link IndexTask.ShardSpecs} representing a map of intervals and corresponding shard specs - */ - private IndexTask.ShardSpecs determineShardSpecs( - final FirehoseFactory firehoseFactory, - final File firehoseTempDir - ) throws IOException - { - final IndexTask.IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); - - final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - - // Must determine intervals if unknown, since we acquire all locks before processing any data. - final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent(); - - // Must determine partitions if # of shards is not provided. - final boolean determineNumPartitions = tuningConfig.getNumShards() == null; - - // if we were given number of shards per interval and the intervals, we don't need to scan the data - if (!determineNumPartitions && !determineIntervals) { - log.info("Skipping determine partition scan"); - return createShardSpecWithoutInputScan(granularitySpec); - } else { - // determine intervals containing data - return createShardSpecsFromInput( - ingestionSchema, - firehoseFactory, - firehoseTempDir, - granularitySpec, - determineIntervals - ); - } - } - - private static IndexTask.ShardSpecs createShardSpecWithoutInputScan(GranularitySpec granularitySpec) - { - final Map> shardSpecs = new HashMap<>(); - final SortedSet intervals = granularitySpec.bucketIntervals().get(); - - for (Interval interval : intervals) { - shardSpecs.put(interval, ImmutableList.of()); - } - - return new IndexTask.ShardSpecs(shardSpecs); - } - - private static IndexTask.ShardSpecs createShardSpecsFromInput( - SinglePhaseParallelIndexIngestionSpec ingestionSchema, - FirehoseFactory firehoseFactory, - File firehoseTempDir, - GranularitySpec granularitySpec, - boolean determineIntervals - ) throws IOException - { - log.info("Determining intervals"); - long determineShardSpecsStartMillis = System.currentTimeMillis(); - - final List intervals = collectIntervalsAndShardSpecs( - ingestionSchema, - firehoseFactory, - firehoseTempDir, - granularitySpec, - determineIntervals - ); - - final Map> intervalToShardSpecs = intervals - .stream() - .collect(Collectors.toMap(Function.identity(), i -> ImmutableList.of())); - log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); - - return new IndexTask.ShardSpecs(intervalToShardSpecs); - } - - private static List collectIntervalsAndShardSpecs( - SinglePhaseParallelIndexIngestionSpec ingestionSchema, - FirehoseFactory firehoseFactory, - File firehoseTempDir, - GranularitySpec granularitySpec, - boolean determineIntervals - ) throws IOException - { - final List intervals = new ArrayList<>(); - int thrownAway = 0; - int unparseable = 0; - - try ( - final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser(), firehoseTempDir) - ) { - while (firehose.hasMore()) { - try { - final InputRow inputRow = firehose.nextRow(); - - // The null inputRow means the caller must skip this row. - if (inputRow == null) { - continue; - } - - final Interval interval; - if (determineIntervals) { - interval = granularitySpec.getSegmentGranularity().bucket(inputRow.getTimestamp()); - } else { - final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); - if (!optInterval.isPresent()) { - thrownAway++; - continue; - } - interval = optInterval.get(); - } - - if (!intervals.contains(interval)) { - intervals.add(interval); - } - } - catch (ParseException e) { - if (ingestionSchema.getTuningConfig().isReportParseExceptions()) { - throw e; - } else { - unparseable++; - } - } - } - } - - // These metrics are reported in generateAndPushSegments() - if (thrownAway > 0) { - log.warn("Unable to find a matching interval for [%,d] events", thrownAway); - } - if (unparseable > 0) { - log.warn("Unable to parse [%,d] events", unparseable); - } - - intervals.sort(Comparators.intervalsByStartThenEnd()); - return intervals; - } - /** * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}. * If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs @@ -440,8 +270,7 @@ private static List collectIntervalsAndShardSpecs( private List generateAndPushSegments( final TaskToolbox toolbox, final DataSchema dataSchema, - final ShardSpecs shardSpecs, - final Map versions, + @Nullable final Map versions, final FirehoseFactory firehoseFactory, final File firehoseTempDir ) throws IOException, InterruptedException @@ -464,16 +293,31 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final long pushTimeout = tuningConfig.getPushTimeout(); + final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); final SegmentAllocator segmentAllocator; - if (ioConfig.isAppendToExisting()) { - segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema); + if (ioConfig.isAppendToExisting() || !explicitIntervals) { + segmentAllocator = new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SurrogateAction<>( + supervisorTaskId, + new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ) + ); } else { segmentAllocator = new CountingActionBasedSegmentAllocator( toolbox.getTaskActionClient(), getDataSource(), granularitySpec, - shardSpecs.getMap(), versions ); } @@ -504,10 +348,12 @@ dataSchema, new RealtimeIOConfig(null, null, null), null throw new ParseException(errorMsg); } - final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); - if (!optInterval.isPresent()) { - fireDepartmentMetrics.incrementThrownAway(); - continue; + if (explicitIntervals) { + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + if (!optInterval.isPresent()) { + fireDepartmentMetrics.incrementThrownAway(); + continue; + } } // Segments are created as needed, using a single sequence name. They may be allocated from the overlord @@ -547,7 +393,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null return pushedSegments; } catch (TimeoutException | ExecutionException e) { - throw Throwables.propagate(e); + throw new RuntimeException(e); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java index 9f83e7ba3893..2cb8d8a1ee08 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java @@ -132,12 +132,6 @@ private CountingSegmentAllocateAction createAction(DateTime timestamp) Granularities.DAY, ImmutableList.of(Intervals.of("2017-01-01/2017-01-02"), Intervals.of("2017-01-02/2017-01-03")) ), - ImmutableMap.of( - Intervals.of("2017-01-01/2017-01-02"), - ImmutableList.of(new NumberedShardSpec(0, 0)), - Intervals.of("2017-01-02/2017-01-03"), - ImmutableList.of(new NumberedShardSpec(1, 0)) - ), ImmutableMap.of( Intervals.of("2017-01-01/2017-01-02"), "version1", diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java new file mode 100644 index 000000000000..9311c92aba89 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.actions; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.indexing.common.TaskLock; +import io.druid.indexing.common.TaskLockType; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.Intervals; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class SurrogateActionTest +{ + @Test + public void testSerde() throws IOException + { + final ObjectMapper objectMapper = new DefaultObjectMapper(); + final SurrogateAction surrogateAction = new SurrogateAction<>( + "testId", new LockTryAcquireAction(TaskLockType.EXCLUSIVE, Intervals.of("2018-01-01/2019-01-01")) + ); + + final String json = objectMapper.writeValueAsString(surrogateAction); + Assert.assertEquals(surrogateAction.toString(), objectMapper.readValue(json, TaskAction.class).toString()); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java index 645382ed3990..9824e04310ee 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java @@ -30,7 +30,8 @@ import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.SurrogateLockListAction; +import io.druid.indexing.common.actions.LockListAction; +import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.SubTaskStateResponse; import io.druid.java.util.common.DateTimes; @@ -622,7 +623,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception final TestFirehose firehose = (TestFirehose) getIngestionSchema().getIOConfig().getFirehoseFactory(); final List locks = toolbox.getTaskActionClient() - .submit(new SurrogateLockListAction(getSupervisorTaskId())); + .submit(new SurrogateAction<>(getSupervisorTaskId(), new LockListAction())); Preconditions.checkState(locks.size() == 1, "There should be a single lock"); task.collectReport( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java index 3110648a8638..ad668453bd1f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java @@ -61,6 +61,7 @@ public void setup() throws IOException for (int i = 0; i < 5; i++) { try (final Writer writer = Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 24 + i, i)); writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 25 + i, i)); } } From 185549f0e8ef27bc7c5678a8b4f6b72d1d0ab3d1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Apr 2018 18:16:21 -0700 Subject: [PATCH 21/47] address comments --- .../development/extensions-contrib/azure.md | 1 + .../extensions-contrib/cloudfiles.md | 1 + .../development/extensions-contrib/google.md | 1 + .../content/development/extensions-core/s3.md | 1 + docs/content/ingestion/firehose.md | 2 + docs/content/ingestion/native_tasks.md | 60 +++--- .../CountingActionBasedSegmentAllocator.java | 3 + .../CountingSegmentAllocateAction.java | 5 + .../common/actions/SurrogateAction.java | 3 + ...inglePhaseParallelIndexSupervisorTask.java | 179 +++++++++++------- .../SinglePhaseParallelIndexTuningConfig.java | 33 +--- .../indexing/common/task/TaskMonitor.java | 31 +-- .../overlord/http/OverlordResource.java | 58 +++--- ...rallelIndexSupervisorTaskResourceTest.java | 48 ++--- 14 files changed, 221 insertions(+), 205 deletions(-) diff --git a/docs/content/development/extensions-contrib/azure.md b/docs/content/development/extensions-contrib/azure.md index 48b739941e54..dba95a822b4f 100644 --- a/docs/content/development/extensions-contrib/azure.md +++ b/docs/content/development/extensions-contrib/azure.md @@ -34,6 +34,7 @@ The storage account is shared with the one used for Azure deep storage functiona As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. Sample spec: diff --git a/docs/content/development/extensions-contrib/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md index f5691b7eac4d..5323b138804e 100644 --- a/docs/content/development/extensions-contrib/cloudfiles.md +++ b/docs/content/development/extensions-contrib/cloudfiles.md @@ -33,6 +33,7 @@ The storage account is shared with the one used for Racksapce's Cloud Files deep As with the Azure blobstore, it is assumed to be gzipped if the extension ends in .gz This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. Sample spec: diff --git a/docs/content/development/extensions-contrib/google.md b/docs/content/development/extensions-contrib/google.md index 2ea55c02293f..1c6ad8bacede 100644 --- a/docs/content/development/extensions-contrib/google.md +++ b/docs/content/development/extensions-contrib/google.md @@ -24,6 +24,7 @@ This firehose ingests events, similar to the StaticS3Firehose, but from an Googl As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. Sample spec: diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index dbf2b553c59b..72f68447d051 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -29,6 +29,7 @@ S3-compatible deep storage is basically either S3 or something like Google Stora This firehose ingests events from a predefined list of S3 objects. This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. Sample spec: diff --git a/docs/content/ingestion/firehose.md b/docs/content/ingestion/firehose.md index d974b74ab167..486e0fc89576 100644 --- a/docs/content/ingestion/firehose.md +++ b/docs/content/ingestion/firehose.md @@ -21,6 +21,7 @@ For additional firehoses, please see our [extensions list](../development/extens This Firehose can be used to read the data from files on local disk. It can be used for POCs to ingest data on disk. This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#single-phase-parallel-index-task). +Since each split represents a file in this firehose, each worker task of `index_single_phase_parallel` will read a file. A sample local firehose spec is shown below: ```json @@ -41,6 +42,7 @@ A sample local firehose spec is shown below: This Firehose can be used to read the data from remote sites via HTTP. This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#single-phase-parallel-index-task). +Since each split represents a file in this firehose, each worker task of `index_single_phase_parallel` will read a file. A sample http firehose spec is shown below: ```json diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 1def6e2a75a1..92b9acb3c83f 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -3,7 +3,9 @@ layout: doc_page --- # Native Index Tasks -Druid currently has two types of native index tasks, i.e., `local index task` and `single phase parallel index task`. +Druid currently has two types of native batch indexing tasks, `index_single_phase_parallel` which runs tasks +in parallel on multiple middle manager nodes, and `index` which will run a single indexing task locally on a single +middle manager. Single Phase Parallel Index Task -------------------------------- @@ -18,8 +20,8 @@ If all worker tasks succeed, then it collects the reported list of generated seg To use this task, the `firehose` in `ioConfig` should be _splittable_. If it's not, this task runs sequentially. The current splittable fireshoses are [`LocalFirehose`](./firehose.html#localfirehose), [`HttpFirehose`](./firehose.html#httpfirehose) -, [`StaticS3Firehose`](../development/extensions-core/statics3firehose), [`StaticAzureBlobStoreFirehose`](../development/extensions-contrib/staticazureblobstorefirehose) -, [`StaticGoogleBlobStoreFirehose`](../development/extensions-contrib/staticgoogleblobstorefirehose), and [`StaticCloudFilesFirehose`](../development/extensions-contrib/staticcloudfilesfirehose). +, [`StaticS3Firehose`](../development/extensions-core/s3.html#statics3firehose), [`StaticAzureBlobStoreFirehose`](../development/extensions-contrib/azure.html#staticazureblobstorefirehose) +, [`StaticGoogleBlobStoreFirehose`](../development/extensions-contrib/google.html#staticgoogleblobstorefirehose), and [`StaticCloudFilesFirehose`](../development/extensions-contrib/cloudfiles.html#staticcloudfilesfirehose). The splittable firehose is responsible for generating _splits_. The supervisor task generates _worker task specs_ each of which specifies a split and submits worker tasks using those specs. As a result, the number of worker tasks depends on @@ -149,53 +151,37 @@ The tuningConfig is optional and default parameters will be used if no tuningCon The supervisor task provides some HTTP endpoints to get running status. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/isRunningInParallel` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}//mode` -Returns the indexing task is running in parallel. It returns false if the firehose in IOConfig is not splittable. +Returns 'parallel' if the indexing task is running in parallel. Otherwise, it returns 'sequential'. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numRunningTasks` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/status` -Returns the number of running worker tasks or 0 if isRunningInParallel is false. +Returns the current running status if the supervisor task is running in the parallel mode. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numSucceededTasks` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtasks/running` -Returns the number of succeeded worker tasks or 0 if isRunningInParallel is false. +Returns the task IDs of running worker tasks, or an empty list if the supervisor task is running in the sequential mode. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numFailedTasks` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs` -Returns the number of failed worker tasks or 0 if isRunningInParallel is false. +Returns all worker task specs, or an empty list if the supervisor task is running in the sequential mode. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/numCompleteTasks` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/running` -Returns the number of succeeded or failed worker tasks, or 0 if isRunningInParallel is false. +Returns running worker task specs, or an empty list if the supervisor task is running in the sequential mode. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/expectedNumSucceededTasks` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/complete` -Returns the expected number of succeeded worker tasks to complete, or 0 if isRunningInParallel is false. +Returns complete worker task specs, or an empty list if the supervisor task is running in the sequential mode. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/runningSubTasks` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}` -Returns the task IDs of running worker tasks, or an empty list if isRunningInParallel is false. +Returns the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subTaskSpecs` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/state` -Returns all worker task specs, or an empty list if isRunningInParallel is false. - -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/runningSubTaskSpecs` - -Returns running worker task specs, or an empty list if isRunningInParallel is false. - -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/completeSubTaskSpecs` - -Returns complete worker task specs, or an empty list if isRunningInParallel is false. - -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subTaskSpec?id={SUB_TASK_SPEC_ID}` - -Returns the worker task spec of the given id, or HTTP 404 Not Found error if isRunningInParallel is false. - -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subTaskState?id={SUB_TASK_SPEC_ID}` - -Returns the state of the worker task spec of the given id, or HTTP 404 Not Found error if isRunningInParallel is false. +Returns the state of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode. The returned result contains the worker task spec, a current task status if exists, and task attempt history. An example of the result is @@ -365,9 +351,9 @@ An example of the result is } ``` -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/completeSubTaskSpecAttemptHistory?id={SUB_TASK_SPEC_ID}` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/history` -Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if isRunningInParallel is false. +Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode. Local Index Task ---------------- diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java index b9956b64d0f2..77c2b901b075 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java @@ -31,6 +31,9 @@ import java.io.IOException; import java.util.Map; +/** + * Segment allocator based on {@link CountingSegmentAllocateAction}. + */ public class CountingActionBasedSegmentAllocator implements SegmentAllocator { private final TaskActionClient taskActionClient; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java index 88f634b6e0ba..2d1a7b8a497b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java @@ -42,6 +42,11 @@ import java.util.Objects; import java.util.SortedSet; +/** + * This action is to find a proper {@link io.druid.timeline.partition.ShardSpec} based on counting. This keeps track of + * the next shard number per {@link Interval} in {@link Counters}. The next shard number is incremented by 1 whenever a + * new {@link SegmentIdentifier} is allocated. + */ public class CountingSegmentAllocateAction implements TaskAction { private final String dataSource; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java index 01b98b2cc0b5..35efae1af772 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java @@ -25,6 +25,9 @@ import io.druid.indexing.common.task.Task; import io.druid.java.util.common.ISE; +/** + * Perform the given action using {@link #surrogateId} on behalf of the caller task. + */ public class SurrogateAction> implements TaskAction { private final String surrogateId; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java index f2508d979c7a..71f2575e37da 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java @@ -66,12 +66,11 @@ import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; +import javax.ws.rs.PathParam; import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -279,11 +278,12 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception state = TaskState.SUCCESS; } else { // Failed + final Status monitorStatus = taskMonitor.getStatus(); throw new ISE( "Expected for [%d] tasks to succeed, but we got [%d] succeeded tasks and [%d] failed tasks", - taskMonitor.getExpectedNumSucceededTasks(), - taskMonitor.getNumSucceededTasks(), - taskMonitor.getNumFailedTasks() + monitorStatus.expectedSucceeded, + monitorStatus.succeeded, + monitorStatus.failed ); } } @@ -376,9 +376,20 @@ public Response report( @VisibleForTesting void collectReport(PushedSegmentsReport report) { - if (segmentsMap.put(report.getTaskId(), report) != null) { - throw new ISE("Dupliate task report from task[%s]", report.getTaskId()); - } + // subTasks might send their reports multiple times because of the HTTP retry. + // Here, we simply make sure the current report is exactly same with the previous one. + segmentsMap.compute(report.getTaskId(), (taskId, prevReport) -> { + if (prevReport != null) { + Preconditions.checkState( + prevReport.getSegments().equals(report.getSegments()), + "task[%s] sent two or more reports and previous report[%s] is different from the current one[%s]", + taskId, + prevReport, + report + ); + } + return report; + }); } private static IndexTuningConfig convertToIndexTuningConfig(SinglePhaseParallelIndexTuningConfig tuningConfig) @@ -511,66 +522,25 @@ private static List getSplitsIfSplittable(FirehoseFactory firehoseFa // External APIs to get running status @GET - @Path("isRunningInParallel") + @Path("/mode") @Produces(MediaType.APPLICATION_JSON) public Response isRunningInParallel(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - return Response.ok(baseFirehoseFactory.isSplittable()).build(); - } - - @GET - @Path("/numRunningTasks") - @Produces(MediaType.APPLICATION_JSON) - public Response getNumRunningTasks(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final int numRunningTasks = taskMonitor == null ? 0 : taskMonitor.getNumRunningTasks(); - return Response.ok(numRunningTasks).build(); - } - - @GET - @Path("/numSucceededTasks") - @Produces(MediaType.APPLICATION_JSON) - public Response getNumSucceededTasks(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final int numSucceededTasks = taskMonitor == null ? 0 : taskMonitor.getNumSucceededTasks(); - return Response.ok(numSucceededTasks).build(); + return Response.ok(baseFirehoseFactory.isSplittable() ? "parallel" : "sequential").build(); } @GET - @Path("/numFailedTasks") + @Path("/status") @Produces(MediaType.APPLICATION_JSON) - public Response getNumFailedTasks(@Context final HttpServletRequest req) + public Response getStatus(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final int numFailedTasks = taskMonitor == null ? 0 : taskMonitor.getNumFailedTasks(); - return Response.ok(numFailedTasks).build(); + return Response.ok(taskMonitor == null ? Status.empty() : taskMonitor.getStatus()).build(); } @GET - @Path("/numCompleteTasks") - @Produces(MediaType.APPLICATION_JSON) - public Response getNumCompleteTasks(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final int numCompleteTasks = taskMonitor == null ? 0 : taskMonitor.getNumCompleteTasks(); - return Response.ok(numCompleteTasks).build(); - } - - @GET - @Path("/expectedNumSucceededTasks") - @Produces(MediaType.APPLICATION_JSON) - public Response getExpectedNumSucceededTasks(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final int expectedNumSucceededTasks = taskMonitor == null ? 0 : taskMonitor.getExpectedNumSucceededTasks(); - return Response.ok(expectedNumSucceededTasks).build(); - } - - @GET - @Path("/runningSubTasks") + @Path("/subtasks/running") @Produces(MediaType.APPLICATION_JSON) public Response getRunningTasks(@Context final HttpServletRequest req) { @@ -580,7 +550,7 @@ public Response getRunningTasks(@Context final HttpServletRequest req) } @GET - @Path("/subTaskSpecs") + @Path("/subtaskspecs") @Produces(MediaType.APPLICATION_JSON) public Response getSubTaskSpecs(@Context final HttpServletRequest req) { @@ -603,7 +573,7 @@ public Response getSubTaskSpecs(@Context final HttpServletRequest req) } @GET - @Path("/runningSubTaskSpecs") + @Path("/subtaskspecs/running") @Produces(MediaType.APPLICATION_JSON) public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) { @@ -615,7 +585,7 @@ public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) } @GET - @Path("/completeSubTaskSpecs") + @Path("/subtaskspecs/complete") @Produces(MediaType.APPLICATION_JSON) public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) { @@ -628,9 +598,9 @@ public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) } @GET - @Path("/subTaskSpec") + @Path("/subtaskspec/{id}") @Produces(MediaType.APPLICATION_JSON) - public Response getSubTaskSpec(@QueryParam("id") String id, @Context final HttpServletRequest req) + public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (taskMonitor != null) { @@ -651,23 +621,23 @@ public Response getSubTaskSpec(@QueryParam("id") String id, @Context final HttpS } if (subTaskSpec == null) { - return Response.status(Status.NOT_FOUND).build(); + return Response.status(Response.Status.NOT_FOUND).build(); } else { return Response.ok(subTaskSpec).build(); } } else { - return Response.status(Status.NOT_FOUND).build(); + return Response.status(Response.Status.NOT_FOUND).build(); } } @GET - @Path("/subTaskState") + @Path("/subtaskspec/{id}/state") @Produces(MediaType.APPLICATION_JSON) - public Response getSubTaskState(@QueryParam("id") String id, @Context final HttpServletRequest req) + public Response getSubTaskState(@PathParam("id") String id, @Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (taskMonitor == null) { - return Response.status(Status.NOT_FOUND).build(); + return Response.status(Response.Status.NOT_FOUND).build(); } else { // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after // adding them to taskHistory. @@ -695,7 +665,7 @@ public Response getSubTaskState(@QueryParam("id") String id, @Context final Http } if (subTaskStateResponse == null) { - return Response.status(Status.NOT_FOUND).build(); + return Response.status(Response.Status.NOT_FOUND).build(); } else { return Response.ok(subTaskStateResponse).build(); } @@ -703,27 +673,96 @@ public Response getSubTaskState(@QueryParam("id") String id, @Context final Http } @GET - @Path("/completeSubTaskSpecAttemptHistory") + @Path("/subtaskspec/{id}/history") @Produces(MediaType.APPLICATION_JSON) public Response getCompleteSubTaskSpecAttemptHistory( - @QueryParam("id") String id, + @PathParam("id") String id, @Context final HttpServletRequest req ) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (taskMonitor == null) { - return Response.status(Status.NOT_FOUND).build(); + return Response.status(Response.Status.NOT_FOUND).build(); } else { final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); if (taskHistory == null) { - return Response.status(Status.NOT_FOUND).build(); + return Response.status(Response.Status.NOT_FOUND).build(); } else { return Response.ok(taskHistory.getAttemptHistory()).build(); } } } + static class Status + { + private final int running; + private final int succeeded; + private final int failed; + private final int complete; + private final int total; + private final int expectedSucceeded; + + static Status empty() + { + return new Status(0, 0, 0, 0, 0, 0); + } + + @JsonCreator + Status( + @JsonProperty("running") int running, + @JsonProperty("succeeded") int succeeded, + @JsonProperty("failed") int failed, + @JsonProperty("complete") int complete, + @JsonProperty("total") int total, + @JsonProperty("expectedSucceeded") int expectedSucceeded + ) + { + this.running = running; + this.succeeded = succeeded; + this.failed = failed; + this.complete = complete; + this.total = total; + this.expectedSucceeded = expectedSucceeded; + } + + @JsonProperty + public int getRunning() + { + return running; + } + + @JsonProperty + public int getSucceeded() + { + return succeeded; + } + + @JsonProperty + public int getFailed() + { + return failed; + } + + @JsonProperty + public int getComplete() + { + return complete; + } + + @JsonProperty + public int getTotal() + { + return total; + } + + @JsonProperty + public int getExpectedSucceeded() + { + return expectedSucceeded; + } + } + static class SubTaskStateResponse { private final SinglePhaseParallelIndexSubTaskSpec spec; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java index 3fd77555dd3e..b5a53916f5e8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java @@ -163,40 +163,23 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } + if (!super.equals(o)) { + return false; + } SinglePhaseParallelIndexTuningConfig that = (SinglePhaseParallelIndexTuningConfig) o; - return getMaxRowsInMemory() == that.getMaxRowsInMemory() && - Objects.equals(getMaxTotalRows(), that.getMaxTotalRows()) && - getMaxPendingPersists() == that.getMaxPendingPersists() && - isForceExtendableShardSpecs() == that.isForceExtendableShardSpecs() && - isReportParseExceptions() == that.isReportParseExceptions() && - getPushTimeout() == that.getPushTimeout() && - Objects.equals(getTargetPartitionSize(), that.getTargetPartitionSize()) && - Objects.equals(getNumShards(), that.getNumShards()) && - Objects.equals(getIndexSpec(), that.getIndexSpec()) && - Objects.equals(getBasePersistDirectory(), that.getBasePersistDirectory()) && - Objects.equals(getSegmentWriteOutMediumFactory(), that.getSegmentWriteOutMediumFactory()) && - maxNumBatchTasks == that.maxNumBatchTasks && + return maxNumBatchTasks == that.maxNumBatchTasks && maxRetry == that.maxRetry && taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs && - chatHandlerTimeout.equals(that.chatHandlerTimeout) && - chatHandlerNumRetries == that.chatHandlerNumRetries; + chatHandlerNumRetries == that.chatHandlerNumRetries && + Objects.equals(chatHandlerTimeout, that.chatHandlerTimeout); } @Override public int hashCode() { + return Objects.hash( - getTargetPartitionSize(), - getMaxRowsInMemory(), - getMaxTotalRows(), - getNumShards(), - getIndexSpec(), - getBasePersistDirectory(), - getMaxPendingPersists(), - isForceExtendableShardSpecs(), - isReportParseExceptions(), - getPushTimeout(), - getSegmentWriteOutMediumFactory(), + super.hashCode(), maxNumBatchTasks, maxRetry, taskStatusCheckPeriodMs, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index 81ce79dd946e..d6d5176ad372 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -28,6 +28,7 @@ import io.druid.client.indexing.TaskStatusResponse; import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.Status; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; @@ -258,11 +259,6 @@ boolean isSucceeded() } } - int getExpectedNumSucceededTasks() - { - return expectedNumSucceededTasks; - } - int getNumRunningTasks() { synchronized (taskCountLock) { @@ -270,24 +266,17 @@ int getNumRunningTasks() } } - int getNumCompleteTasks() - { - synchronized (taskCountLock) { - return numSucceededTasks + numFailedTasks; - } - } - - int getNumSucceededTasks() + Status getStatus() { synchronized (taskCountLock) { - return numSucceededTasks; - } - } - - int getNumFailedTasks() - { - synchronized (taskCountLock) { - return numFailedTasks; + return new Status( + numRunningTasks, + numSucceededTasks, + numFailedTasks, + numSucceededTasks + numFailedTasks, + numRunningTasks + numSucceededTasks + numFailedTasks, + expectedNumSucceededTasks + ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 622ff799595b..6507ece70cec 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -242,34 +242,36 @@ public Response getTaskStatus(@PathParam("taskid") String taskid) final Pair createdDateAndDataSource = taskStorageQueryAdapter.getCreatedDateAndDataSource( taskid ); - final TaskStatusResponse response; - if (taskMaster.getTaskRunner().isPresent() && - task != null && - taskStatus != null && - createdDateAndDataSource != null) { - final TaskRunner taskRunner = taskMaster.getTaskRunner().get(); - final TaskRunnerWorkItem workItem = taskRunner - .getKnownTasks() - .stream() - .filter(item -> item.getTaskId().equals(taskid)) - .findAny() - .orElse(null); - if (workItem != null) { - response = new TaskStatusResponse( - taskid, - new TaskStatusPlus( - taskid, - task.getType(), - createdDateAndDataSource.lhs, - workItem.getQueueInsertionTime(), - taskStatus.getStatusCode(), - taskStatus.getDuration(), - workItem.getLocation(), - createdDateAndDataSource.rhs, - null - ) - ); - } else { + TaskStatusResponse response = null; + + if (task != null && taskStatus != null && createdDateAndDataSource != null) { + if (taskMaster.getTaskRunner().isPresent()) { + final TaskRunner taskRunner = taskMaster.getTaskRunner().get(); + final TaskRunnerWorkItem workItem = taskRunner + .getKnownTasks() + .stream() + .filter(item -> item.getTaskId().equals(taskid)) + .findAny() + .orElse(null); + if (workItem != null) { + response = new TaskStatusResponse( + taskid, + new TaskStatusPlus( + taskid, + task.getType(), + createdDateAndDataSource.lhs, + workItem.getQueueInsertionTime(), + taskStatus.getStatusCode(), + taskStatus.getDuration(), + workItem.getLocation(), + createdDateAndDataSource.rhs, + null + ) + ); + } + } + + if (response == null) { response = new TaskStatusResponse( taskid, new TaskStatusPlus( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java index 9824e04310ee..ccf010b275bf 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java @@ -33,6 +33,7 @@ import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.Status; import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.SubTaskStateResponse; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; @@ -135,16 +136,16 @@ public void testAPIs() throws Exception // test isRunningInParallel Response response = task.isRunningInParallel(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertTrue((Boolean) response.getEntity()); + Assert.assertEquals("parallel", response.getEntity()); // test expectedNumSucceededTasks - response = task.getExpectedNumSucceededTasks(newRequest()); + response = task.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(NUM_SUB_TASKS, response.getEntity()); + Assert.assertEquals(NUM_SUB_TASKS, ((Status) response.getEntity()).getExpectedSucceeded()); // Since taskMonitor works based on polling, it's hard to use a fancier way to check its state. // We use polling to check the state of taskMonitor in this test. - while (getNumSubTasks(req -> task.getNumRunningTasks(req)) < NUM_SUB_TASKS) { + while (getNumSubTasks(Status::getRunning) < NUM_SUB_TASKS) { Thread.sleep(100); } @@ -163,7 +164,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks(req -> task.getNumSucceededTasks(req)) < succeededTasks) { + while (getNumSubTasks(Status::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -180,7 +181,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.FAILED); } - while (getNumSubTasks(req -> task.getNumFailedTasks(req)) < failedTasks) { + while (getNumSubTasks(Status::getFailed) < failedTasks) { Thread.sleep(100); } @@ -202,7 +203,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks(req -> task.getNumSucceededTasks(req)) < succeededTasks) { + while (getNumSubTasks(Status::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -222,7 +223,7 @@ public void testAPIs() throws Exception // Test one more failure runningTasks.get(0).setState(TaskState.FAILED); failedTasks++; - while (getNumSubTasks(req -> task.getNumFailedTasks(req)) < failedTasks) { + while (getNumSubTasks(Status::getFailed) < failedTasks) { Thread.sleep(100); } @@ -236,7 +237,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); succeededTasks++; - while (getNumSubTasks(req -> task.getNumSucceededTasks(req)) < succeededTasks) { + while (getNumSubTasks(Status::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -244,11 +245,11 @@ public void testAPIs() throws Exception } @SuppressWarnings({"ConstantConditions"}) - private int getNumSubTasks(Function func) + private int getNumSubTasks(Function func) { - final Response response = func.apply(newRequest()); + final Response response = task.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); - return (Integer) response.getEntity(); + return func.apply((Status) response.getEntity()); } private Map buildStateMap() @@ -275,25 +276,24 @@ private void checkState( Map expectedSubTaskStateResponses // subTaskSpecId -> response ) { - // numRunningTasks - Response response = task.getNumRunningTasks(newRequest()); + Response response = task.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(runningTasks.size(), response.getEntity()); + final Status monitorStatus = (Status) response.getEntity(); + + // numRunningTasks + Assert.assertEquals(runningTasks.size(), monitorStatus.getRunning()); // numSucceededTasks - response = task.getNumSucceededTasks(newRequest()); - Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(expectedSucceededTasks, response.getEntity()); + Assert.assertEquals(expectedSucceededTasks, monitorStatus.getSucceeded()); // numFailedTasks - response = task.getNumFailedTasks(newRequest()); - Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(expectedFailedTask, response.getEntity()); + Assert.assertEquals(expectedFailedTask, monitorStatus.getFailed()); // numCompleteTasks - response = task.getNumCompleteTasks(newRequest()); - Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(expectedSucceededTasks + expectedFailedTask, response.getEntity()); + Assert.assertEquals(expectedSucceededTasks + expectedFailedTask, monitorStatus.getComplete()); + + // numTotalTasks + Assert.assertEquals(runningTasks.size() + expectedSucceededTasks + expectedFailedTask, monitorStatus.getTotal()); // runningSubTasks response = task.getRunningTasks(newRequest()); From 9a8ccc4dfd82f7c974dfc0b6c8de80c7a5ec8c7a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Apr 2018 18:31:01 -0700 Subject: [PATCH 22/47] Fix static s3 firehose --- .../firehose/s3/StaticS3FirehoseFactory.java | 68 ++++++++++--------- .../s3/StaticS3FirehoseFactoryTest.java | 2 +- 2 files changed, 36 insertions(+), 34 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 58349a606df3..d68c0f6c04c7 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -57,7 +57,7 @@ /** * Builds firehoses that read from a predefined list of S3 objects and then dry up. */ -public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory +public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); private static final int MAX_LISTING_LENGTH = 1024; @@ -113,20 +113,12 @@ public List getPrefixes() } @Override - protected Collection initObjects() throws IOException + protected Collection initObjects() throws IOException { // Here, the returned s3 objects contain minimal information without data. // Getting data is deferred until openObjectStream() is called for each object. if (!uris.isEmpty()) { - return uris.stream() - .map( - uri -> { - final String s3Bucket = uri.getAuthority(); - final String key = S3Utils.extractS3Key(uri); - return S3Utils.getSingleObjectSummary(s3Client, s3Bucket, key); - } - ) - .collect(Collectors.toList()); + return uris; } else { final List objects = new ArrayList<>(); for (URI uri : prefixes) { @@ -172,18 +164,21 @@ protected Collection initObjects() throws IOException } } } - return objects; + return objects.stream().map(StaticS3FirehoseFactory::toUri).collect(Collectors.toList()); } } @Override - protected InputStream openObjectStream(S3ObjectSummary object) throws IOException + protected InputStream openObjectStream(URI object) throws IOException { try { // Get data of the given object and open an input stream - final S3Object s3Object = s3Client.getObject(object.getBucketName(), object.getKey()); + final String bucket = object.getAuthority(); + final String key = S3Utils.extractS3Key(object); + + final S3Object s3Object = s3Client.getObject(bucket, key); if (s3Object == null) { - throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", object.getBucketName(), object.getKey()); + throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", bucket, key); } return s3Object.getObjectContent(); } @@ -193,17 +188,20 @@ protected InputStream openObjectStream(S3ObjectSummary object) throws IOExceptio } @Override - protected InputStream openObjectStream(S3ObjectSummary object, long start) throws IOException + protected InputStream openObjectStream(URI object, long start) throws IOException { - final GetObjectRequest request = new GetObjectRequest(object.getBucketName(), object.getKey()); + final String bucket = object.getAuthority(); + final String key = S3Utils.extractS3Key(object); + + final GetObjectRequest request = new GetObjectRequest(bucket, key); request.setRange(start); try { final S3Object s3Object = s3Client.getObject(request); if (s3Object == null) { throw new ISE( "Failed to get an s3 object for bucket[%s], key[%s], and start[%d]", - object.getBucketName(), - object.getKey(), + bucket, + key, start ); } @@ -215,9 +213,9 @@ protected InputStream openObjectStream(S3ObjectSummary object, long start) throw } @Override - protected InputStream wrapObjectStream(S3ObjectSummary object, InputStream stream) throws IOException + protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException { - return CompressionUtils.decompress(stream, object.getKey()); + return CompressionUtils.decompress(stream, S3Utils.extractS3Key(object)); } @Override @@ -262,21 +260,11 @@ protected Predicate getRetryCondition() } @Override - public FiniteFirehoseFactory withSplit(InputSplit split) + public FiniteFirehoseFactory withSplit(InputSplit split) { - final String originalAuthority = split.get().getBucketName(); - final String originalPath = split.get().getKey(); - final String authority = originalAuthority.endsWith("/") ? - originalAuthority.substring(0, originalAuthority.length() - 1) : - originalAuthority; - final String path = originalPath.startsWith("/") ? - originalPath.substring(1, originalPath.length()) : - originalPath; - - final URI splitUri = URI.create(StringUtils.format("s3://%s/%s", authority, path)); return new StaticS3FirehoseFactory( s3Client, - Collections.singletonList(splitUri), + Collections.singletonList(split.get()), null, getMaxCacheCapacityBytes(), getMaxFetchCapacityBytes(), @@ -285,4 +273,18 @@ public FiniteFirehoseFactory withSplit(In getMaxFetchRetry() ); } + + private static URI toUri(S3ObjectSummary object) + { + final String originalAuthority = object.getBucketName(); + final String originalPath = object.getKey(); + final String authority = originalAuthority.endsWith("/") ? + originalAuthority.substring(0, originalAuthority.length() - 1) : + originalAuthority; + final String path = originalPath.startsWith("/") ? + originalPath.substring(1, originalPath.length()) : + originalPath; + + return URI.create(StringUtils.format("s3://%s/%s", authority, path)); + } } diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index 75ee66d59877..e15960a2cbbb 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -108,7 +108,7 @@ public void testWithSplit() throws IOException 100L, 5 ); - final List> subFactories = factory + final List> subFactories = factory .getSplits() .map(factory::withSplit) .sorted(Comparator.comparing(eachFactory -> { From 2447bbf5d3403e7c66a43be43ee49b302bcb9d12 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Apr 2018 23:38:17 -0700 Subject: [PATCH 23/47] Fix test --- .../SinglePhaseParallelIndexSupervisorTaskResourceTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java index ccf010b275bf..5685a0a53bd9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java @@ -226,6 +226,9 @@ public void testAPIs() throws Exception while (getNumSubTasks(Status::getFailed) < failedTasks) { Thread.sleep(100); } + while (getNumSubTasks(Status::getRunning) < 1) { + Thread.sleep(100); + } checkState( task, From ac817c606ca9048ac638b4831260e164a43e8c76 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 30 Apr 2018 13:02:11 -0700 Subject: [PATCH 24/47] fix build --- .../indexing/common/task/SinglePhaseParallelIndexSubTask.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java index e3828b26bb82..282410619933 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java @@ -438,7 +438,8 @@ private static BatchAppenderatorDriver newDriver( return new BatchAppenderatorDriver( appenderator, segmentAllocator, - new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()) + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getDataSegmentKiller() ); } } From 4924fa5cd198dea95474d39bded9dd40eb39c83b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 30 Apr 2018 17:04:31 -0700 Subject: [PATCH 25/47] fix test --- ...SinglePhaseParallelIndexSupervisorTaskTest.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java index 8be59a8bd501..b3ec64bb0330 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java @@ -42,6 +42,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; +import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; @@ -216,7 +217,18 @@ public File getStorageDirectory() }, getObjectMapper() ), - null, + new DataSegmentKiller() + { + @Override + public void kill(DataSegment segment) + { + } + + @Override + public void killAll() + { + } + }, null, null, null, From 0998e71cbc89fa530b1f7282580e098cc0c4b19d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 4 May 2018 13:35:04 -0700 Subject: [PATCH 26/47] fix typo in docs --- docs/content/ingestion/native_tasks.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 7f9d7a61d66c..3ba263d752df 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -151,7 +151,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon The supervisor task provides some HTTP endpoints to get running status. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}//mode` +##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/mode` Returns 'parallel' if the indexing task is running in parallel. Otherwise, it returns 'sequential'. From f3063389abb6fa2b32b9913fc9d53689fd7ef3bf Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 4 May 2018 13:38:08 -0700 Subject: [PATCH 27/47] add missing maxBytesInMemory to doc --- docs/content/ingestion/native_tasks.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 3ba263d752df..ca91a14e10dc 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -133,6 +133,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |type|The task type, this should always be `index_single_phase_parallel`.|none|yes| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no| +|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| |maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| From b077c9230989b27d39692bd8aaa6416c44192cfc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 22 May 2018 08:55:31 +0900 Subject: [PATCH 28/47] address comments --- docs/content/ingestion/native_tasks.md | 18 +++++++++--------- .../common/actions/SurrogateAction.java | 12 ++++++------ 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index ca91a14e10dc..9726225fe019 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -152,35 +152,35 @@ The tuningConfig is optional and default parameters will be used if no tuningCon The supervisor task provides some HTTP endpoints to get running status. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/mode` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/mode` Returns 'parallel' if the indexing task is running in parallel. Otherwise, it returns 'sequential'. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/status` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/status` Returns the current running status if the supervisor task is running in the parallel mode. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtasks/running` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtasks/running` Returns the task IDs of running worker tasks, or an empty list if the supervisor task is running in the sequential mode. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs` Returns all worker task specs, or an empty list if the supervisor task is running in the sequential mode. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/running` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/running` Returns running worker task specs, or an empty list if the supervisor task is running in the sequential mode. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/complete` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/complete` Returns complete worker task specs, or an empty list if the supervisor task is running in the sequential mode. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}` Returns the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode. -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/state` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/state` Returns the state of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode. The returned result contains the worker task spec, a current task status if exists, and task attempt history. @@ -352,7 +352,7 @@ An example of the result is } ``` -##### `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/history` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/history` Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode. diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java index 35efae1af772..21aecabd0735 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java @@ -28,15 +28,15 @@ /** * Perform the given action using {@link #surrogateId} on behalf of the caller task. */ -public class SurrogateAction> implements TaskAction +public class SurrogateAction> implements TaskAction { private final String surrogateId; - private final T taskAction; + private final ActionType taskAction; @JsonCreator public SurrogateAction( @JsonProperty("surrogateId") String surrogateId, - @JsonProperty("taskAction") T taskAction + @JsonProperty("taskAction") ActionType taskAction ) { this.surrogateId = surrogateId; @@ -50,19 +50,19 @@ public String getSurrogateId() } @JsonProperty - public T getTaskAction() + public ActionType getTaskAction() { return taskAction; } @Override - public TypeReference getReturnTypeReference() + public TypeReference getReturnTypeReference() { return taskAction.getReturnTypeReference(); } @Override - public R perform(Task task, TaskActionToolbox toolbox) + public ReturyType perform(Task task, TaskActionToolbox toolbox) { final Optional maybeSurrogateTask = toolbox.getTaskStorage().getTask(surrogateId); if (maybeSurrogateTask.isPresent()) { From 3631e2060437fed87ae6ea490530c15dc872206c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 24 May 2018 12:23:54 +0900 Subject: [PATCH 29/47] fix race in test --- ...lePhaseParallelIndexSupervisorTaskResourceTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java index d6c4cec7353f..4bc6c8b7c704 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java @@ -185,6 +185,11 @@ public void testAPIs() throws Exception Thread.sleep(100); } + // Wait for new tasks to be started + while (runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { + Thread.sleep(100); + } + checkState( task, succeededTasks, @@ -192,11 +197,6 @@ public void testAPIs() throws Exception buildStateMap() ); - // Wait for new tasks to be started - while (runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { - Thread.sleep(100); - } - // Make sure only one subTask is running succeededTasks += 7; for (int i = 0; i < 7; i++) { From acb5305abce7f08cb9abb7a8ace98b9f2a8a33fc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 30 May 2018 17:48:08 -0700 Subject: [PATCH 30/47] fix test --- .../io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java index 53a38f053e00..1abb210bce4f 100644 --- a/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java +++ b/extensions-core/s3-extensions/src/test/java/io/druid/firehose/s3/StaticS3FirehoseFactoryTest.java @@ -57,8 +57,9 @@ */ public class StaticS3FirehoseFactoryTest { + private static final AmazonS3Client S3_ClIENT = EasyMock.createNiceMock(AmazonS3Client.class); private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3( - EasyMock.createNiceMock(AmazonS3Client.class), + S3_ClIENT, new NoopServerSideEncryption() ); @@ -101,7 +102,7 @@ public void testWithSplit() throws IOException uris.sort(Comparator.comparing(URI::toString)); uris.forEach(StaticS3FirehoseFactoryTest::addExpectedObjject); - EasyMock.replay(SERVICE); + EasyMock.replay(S3_ClIENT); final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory( SERVICE, From 28e310d8d7532e8c9785f4f25c33135f92eec4d5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 1 Jun 2018 13:18:41 -0700 Subject: [PATCH 31/47] Rename to ParallelIndexSupervisorTask --- .../development/extensions-contrib/azure.md | 4 +- .../extensions-contrib/cloudfiles.md | 4 +- .../development/extensions-contrib/google.md | 4 +- .../content/development/extensions-core/s3.md | 4 +- docs/content/ingestion/firehose.md | 8 +- docs/content/ingestion/native_tasks.md | 40 +-- ...Config.java => ParallelIndexIOConfig.java} | 6 +- ...c.java => ParallelIndexIngestionSpec.java} | 19 +- ...SubTask.java => ParallelIndexSubTask.java} | 39 +- ...pec.java => ParallelIndexSubTaskSpec.java} | 14 +- .../task/ParallelIndexSupervisorTask.java | 317 ++++++++++++++++ ...ient.java => ParallelIndexTaskClient.java} | 4 +- ...va => ParallelIndexTaskClientFactory.java} | 9 +- .../common/task/ParallelIndexTaskRunner.java | 32 ++ ...ig.java => ParallelIndexTuningConfig.java} | 30 +- ...> SinglePhaseParallelIndexTaskRunner.java} | 339 ++++-------------- .../io/druid/indexing/common/task/Task.java | 4 +- .../indexing/common/task/TaskMonitor.java | 2 +- ...tractParallelIndexSupervisorTaskTest.java} | 79 ++-- ...allelIndexSupervisorTaskResourceTest.java} | 164 ++++++--- ...ParallelIndexSupervisorTaskSerdeTest.java} | 17 +- ...a => ParallelIndexSupervisorTaskTest.java} | 121 +++++-- ...ndexTest.java => ITParallelIndexTest.java} | 8 +- ... => wikipedia_parallel_index_queries.json} | 4 +- ...son => wikipedia_parallel_index_task.json} | 6 +- .../java/io/druid/cli/CliMiddleManager.java | 4 +- .../main/java/io/druid/cli/CliOverlord.java | 4 +- .../src/main/java/io/druid/cli/CliPeon.java | 8 +- 28 files changed, 801 insertions(+), 493 deletions(-) rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexIOConfig.java => ParallelIndexIOConfig.java} (89%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexIngestionSpec.java => ParallelIndexIngestionSpec.java} (68%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexSubTask.java => ParallelIndexSubTask.java} (91%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexSubTaskSpec.java => ParallelIndexSubTaskSpec.java} (77%) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexTaskClient.java => ParallelIndexTaskClient.java} (95%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexTaskClientFactory.java => ParallelIndexTaskClientFactory.java} (86%) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexTuningConfig.java => ParallelIndexTuningConfig.java} (86%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexSupervisorTask.java => SinglePhaseParallelIndexTaskRunner.java} (66%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{AbstractSinglePhaseParallelIndexSupervisorTaskTest.java => AbstractParallelIndexSupervisorTaskTest.java} (78%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexSupervisorTaskResourceTest.java => ParallelIndexSupervisorTaskResourceTest.java} (80%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexSupervisorTaskSerdeTest.java => ParallelIndexSupervisorTaskSerdeTest.java} (90%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{SinglePhaseParallelIndexSupervisorTaskTest.java => ParallelIndexSupervisorTaskTest.java} (69%) rename integration-tests/src/test/java/io/druid/tests/indexer/{ITSinglePhaseParallelIndexTest.java => ITParallelIndexTest.java} (85%) rename integration-tests/src/test/resources/indexer/{wikipedia_parallel_single_phase_index_queries.json => wikipedia_parallel_index_queries.json} (93%) rename integration-tests/src/test/resources/indexer/{wikipedia_parallel_single_phase_index_task.json => wikipedia_parallel_index_task.json} (92%) diff --git a/docs/content/development/extensions-contrib/azure.md b/docs/content/development/extensions-contrib/azure.md index dba95a822b4f..cb3d1b34fcb1 100644 --- a/docs/content/development/extensions-contrib/azure.md +++ b/docs/content/development/extensions-contrib/azure.md @@ -33,8 +33,8 @@ The storage account is shared with the one used for Azure deep storage functiona As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz -This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). -Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object. Sample spec: diff --git a/docs/content/development/extensions-contrib/cloudfiles.md b/docs/content/development/extensions-contrib/cloudfiles.md index 5323b138804e..ad1c61225058 100644 --- a/docs/content/development/extensions-contrib/cloudfiles.md +++ b/docs/content/development/extensions-contrib/cloudfiles.md @@ -32,8 +32,8 @@ The storage account is shared with the one used for Racksapce's Cloud Files deep As with the Azure blobstore, it is assumed to be gzipped if the extension ends in .gz -This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). -Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object. Sample spec: diff --git a/docs/content/development/extensions-contrib/google.md b/docs/content/development/extensions-contrib/google.md index 1c6ad8bacede..89c35260270f 100644 --- a/docs/content/development/extensions-contrib/google.md +++ b/docs/content/development/extensions-contrib/google.md @@ -23,8 +23,8 @@ This firehose ingests events, similar to the StaticS3Firehose, but from an Googl As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz -This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). -Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object. Sample spec: diff --git a/docs/content/development/extensions-core/s3.md b/docs/content/development/extensions-core/s3.md index df9f3628a61b..5f8a07fef597 100644 --- a/docs/content/development/extensions-core/s3.md +++ b/docs/content/development/extensions-core/s3.md @@ -43,8 +43,8 @@ You can enable [server-side encryption](https://docs.aws.amazon.com/AmazonS3/lat ## StaticS3Firehose This firehose ingests events from a predefined list of S3 objects. -This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#single-phase-parallel-index-task). -Since each split represents an object in this firehose, each worker task of `index_single_phase_parallel` will read an object. +This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task). +Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object. Sample spec: diff --git a/docs/content/ingestion/firehose.md b/docs/content/ingestion/firehose.md index 92286a9aead2..b127631c12ba 100644 --- a/docs/content/ingestion/firehose.md +++ b/docs/content/ingestion/firehose.md @@ -20,8 +20,8 @@ For additional firehoses, please see our [extensions list](../development/extens This Firehose can be used to read the data from files on local disk. It can be used for POCs to ingest data on disk. -This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#single-phase-parallel-index-task). -Since each split represents a file in this firehose, each worker task of `index_single_phase_parallel` will read a file. +This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#parallel-index-task). +Since each split represents a file in this firehose, each worker task of `index_parallel` will read a file. A sample local firehose spec is shown below: ```json @@ -41,8 +41,8 @@ A sample local firehose spec is shown below: #### HttpFirehose This Firehose can be used to read the data from remote sites via HTTP. -This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#single-phase-parallel-index-task). -Since each split represents a file in this firehose, each worker task of `index_single_phase_parallel` will read a file. +This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#parallel-index-task). +Since each split represents a file in this firehose, each worker task of `index_parallel` will read a file. A sample http firehose spec is shown below: ```json diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 9726225fe019..215b34dc7e27 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -3,16 +3,16 @@ layout: doc_page --- # Native Index Tasks -Druid currently has two types of native batch indexing tasks, `index_single_phase_parallel` which runs tasks +Druid currently has two types of native batch indexing tasks, `index_parallel` which runs tasks in parallel on multiple middle manager nodes, and `index` which will run a single indexing task locally on a single middle manager. -Single Phase Parallel Index Task +Parallel Index Task -------------------------------- -The Single Phase Parallel Index Task is a task for parallel batch indexing. This task only uses Druid's resource and -doesn't depend on other external systems like Hadoop. This task works in a single phase without shuffling intermediate -data. `index_single_phase_parallel` task is a supervisor task which basically generates multiple worker tasks and submits +The Parallel Index Task is a task for parallel batch indexing. This task only uses Druid's resource and +doesn't depend on other external systems like Hadoop. This task currently works in a single phase without shuffling intermediate +data. `index_parallel` task is a supervisor task which basically generates multiple worker tasks and submits them to overlords. Each worker task reads input data and makes segments. Once they successfully generate segments for all input, they report the generated segment list to the supervisor task. The supervisor task periodically checks the worker task statuses. If one of them fails, it retries the failed task until the retrying number reaches the configured limit. @@ -34,10 +34,10 @@ An example ingestion spec is: ```json { - "type": "index_single_phase_parallel", + "type": "index_parallel", "spec": { "dataSchema": { - "dataSource": "wikipedia_parallel_single_phase_index_test", + "dataSource": "wikipedia_parallel_index_test", "metricsSpec": [ { "type": "count", @@ -90,7 +90,7 @@ An example ingestion spec is: } }, "ioConfig": { - "type": "index_single_phase_parallel", + "type": "index_parallel", "firehose": { "type": "local", "baseDir": "examples/indexing/", @@ -105,7 +105,7 @@ An example ingestion spec is: |property|description|required?| |--------|-----------|---------| -|type|The task type, this should always be `index_single_phase_parallel`.|yes| +|type|The task type, this should always be `index_parallel`.|yes| |id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no| |spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes| |context|Context containing various task configuration parameters. See below for more details.|no| @@ -120,7 +120,7 @@ See [Ingestion](../ingestion/index.html) |property|description|default|required?| |--------|-----------|-------|---------| -|type|The task type, this should always be `index_single_phase_parallel`.|none|yes| +|type|The task type, this should always be `index_parallel`.|none|yes| |firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes| |appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no| @@ -130,7 +130,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |property|description|default|required?| |--------|-----------|-------|---------| -|type|The task type, this should always be `index_single_phase_parallel`.|none|yes| +|type|The task type, this should always be `index_parallel`.|none|yes| |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no| |maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| @@ -142,7 +142,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| |pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no| |segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no| -|maxNumBatchTasks|Maximum number of tasks which can be run at the same time.|Integer.MAX_VALUE|no| +|maxNumSubTasks|Maximum number of tasks which can be run at the same time.|Integer.MAX_VALUE|no| |maxRetry|Maximum number of retries on task failures.|3|no| |taskStatusCheckPeriodMs|Polling period in milleseconds to check running task statuses.|1000|no| |chatHandlerTimeout|Timeout for reporting the pushed segments in worker tasks.|PT10S|no| @@ -190,9 +190,9 @@ An example of the result is ```json { "spec": { - "id": "index_single_phase_parallel_lineitem_2018-04-20T22:12:43.610Z_2", - "groupId": "index_single_phase_parallel_lineitem_2018-04-20T22:12:43.610Z", - "supervisorTaskId": "index_single_phase_parallel_lineitem_2018-04-20T22:12:43.610Z", + "id": "index_parallel_lineitem_2018-04-20T22:12:43.610Z_2", + "groupId": "index_parallel_lineitem_2018-04-20T22:12:43.610Z", + "supervisorTaskId": "index_parallel_lineitem_2018-04-20T22:12:43.610Z", "context": null, "inputSplit": { "split": "/path/to/data/lineitem.tbl.5" @@ -292,7 +292,7 @@ An example of the result is } }, "ioConfig": { - "type": "index_single_phase_parallel", + "type": "index_parallel", "firehose": { "type": "local", "baseDir": "/path/to/data/", @@ -302,7 +302,7 @@ An example of the result is "appendToExisting": false }, "tuningConfig": { - "type": "index_single_phase_parallel", + "type": "index_parallel", "targetPartitionSize": 5000000, "maxRowsInMemory": 1000000, "maxTotalRows": 20000000, @@ -320,7 +320,7 @@ An example of the result is "reportParseExceptions": false, "pushTimeout": 0, "segmentWriteOutMediumFactory": null, - "maxNumBatchTasks": 2147483647, + "maxNumSubTasks": 2147483647, "maxRetry": 3, "taskStatusCheckPeriodMs": 1000, "chatHandlerTimeout": "PT10S", @@ -334,8 +334,8 @@ An example of the result is } }, "currentStatus": { - "id": "index_single_phase_sub_lineitem_2018-04-20T22:16:29.922Z", - "type": "index_single_phase_sub", + "id": "index_sub_lineitem_2018-04-20T22:16:29.922Z", + "type": "index_sub", "createdTime": "2018-04-20T22:16:29.925Z", "queueInsertionTime": "2018-04-20T22:16:29.929Z", "statusCode": "RUNNING", diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java similarity index 89% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIOConfig.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java index 9ca0686fd5c2..1ef2d62b04b8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIOConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java @@ -29,10 +29,10 @@ /** * Same with {@link IndexIOConfig} except its JSON type name. */ -@JsonTypeName("index_single_phase_parallel") -public class SinglePhaseParallelIndexIOConfig extends IndexIOConfig +@JsonTypeName("index_parallel") +public class ParallelIndexIOConfig extends IndexIOConfig { - public SinglePhaseParallelIndexIOConfig( + public ParallelIndexIOConfig( @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting ) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIngestionSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java similarity index 68% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIngestionSpec.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java index c17824213369..bf76ef6dc7f1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexIngestionSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java @@ -24,25 +24,24 @@ import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.IngestionSpec; -public class SinglePhaseParallelIndexIngestionSpec - extends IngestionSpec +public class ParallelIndexIngestionSpec extends IngestionSpec { private final DataSchema dataSchema; - private final SinglePhaseParallelIndexIOConfig ioConfig; - private final SinglePhaseParallelIndexTuningConfig tuningConfig; + private final ParallelIndexIOConfig ioConfig; + private final ParallelIndexTuningConfig tuningConfig; @JsonCreator - public SinglePhaseParallelIndexIngestionSpec( + public ParallelIndexIngestionSpec( @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") SinglePhaseParallelIndexIOConfig ioConfig, - @JsonProperty("tuningConfig") SinglePhaseParallelIndexTuningConfig tuningConfig + @JsonProperty("ioConfig") ParallelIndexIOConfig ioConfig, + @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig ) { super(dataSchema, ioConfig, tuningConfig); this.dataSchema = dataSchema; this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig == null ? SinglePhaseParallelIndexTuningConfig.defaultConfig() : tuningConfig; + this.tuningConfig = tuningConfig == null ? ParallelIndexTuningConfig.defaultConfig() : tuningConfig; } @Override @@ -54,14 +53,14 @@ public DataSchema getDataSchema() @Override @JsonProperty("ioConfig") - public SinglePhaseParallelIndexIOConfig getIOConfig() + public ParallelIndexIOConfig getIOConfig() { return ioConfig; } @Override @JsonProperty("tuningConfig") - public SinglePhaseParallelIndexTuningConfig getTuningConfig() + public ParallelIndexTuningConfig getTuningConfig() { return tuningConfig; } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java similarity index 91% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java index d392c09f5378..6912a42f0f85 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java @@ -80,33 +80,34 @@ import java.util.stream.Collectors; /** - * A worker task of {@link SinglePhaseParallelIndexSupervisorTask}. Similar to {@link IndexTask}, but this task - * generates and pushes segments, and reports them to the {@link SinglePhaseParallelIndexSupervisorTask} instead of + * A worker task of {@link ParallelIndexSupervisorTask}. Similar to {@link IndexTask}, but this task + * generates and pushes segments, and reports them to the {@link ParallelIndexSupervisorTask} instead of * publishing on its own. */ -public class SinglePhaseParallelIndexSubTask extends AbstractTask +public class ParallelIndexSubTask extends AbstractTask { - private static final Logger log = new Logger(SinglePhaseParallelIndexSubTask.class); - private static final String TYPE = "index_single_phase_sub"; + static final String TYPE = "index_sub"; + + private static final Logger log = new Logger(ParallelIndexSubTask.class); private final int numAttempts; - private final SinglePhaseParallelIndexIngestionSpec ingestionSchema; + private final ParallelIndexIngestionSpec ingestionSchema; private final String supervisorTaskId; private final IndexingServiceClient indexingServiceClient; - private final IndexTaskClientFactory taskClientFactory; + private final IndexTaskClientFactory taskClientFactory; @JsonCreator - public SinglePhaseParallelIndexSubTask( - // id shouldn't be null except when this task is created by SinglePhaseParallelIndexSupervisorTask + public ParallelIndexSubTask( + // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask @JsonProperty("id") @Nullable final String id, @JsonProperty("groupId") final String groupId, @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("supervisorTaskId") final String supervisorTaskId, @JsonProperty("numAttempts") final int numAttempts, // zero-based counting - @JsonProperty("spec") final SinglePhaseParallelIndexIngestionSpec ingestionSchema, + @JsonProperty("spec") final ParallelIndexIngestionSpec ingestionSchema, @JsonProperty("context") final Map context, @JacksonInject IndexingServiceClient indexingServiceClient, - @JacksonInject IndexTaskClientFactory taskClientFactory + @JacksonInject IndexTaskClientFactory taskClientFactory ) { super( @@ -169,7 +170,7 @@ public int getNumAttempts() } @JsonProperty("spec") - public SinglePhaseParallelIndexIngestionSpec getIngestionSchema() + public ParallelIndexIngestionSpec getIngestionSchema() { return ingestionSchema; } @@ -217,7 +218,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception firehoseFactory, firehoseTempDir ); - final SinglePhaseParallelIndexTaskClient taskClient = taskClientFactory.build( + final ParallelIndexTaskClient taskClient = taskClientFactory.build( new ClientBasedTaskInfoProvider(indexingServiceClient), getId(), 1, // always use a single http thread @@ -255,10 +256,10 @@ private Map tryAcquireExclusiveSurrogateLocks( * *

    *
  • - * If the number of rows in a segment exceeds {@link SinglePhaseParallelIndexTuningConfig#targetPartitionSize} + * If the number of rows in a segment exceeds {@link ParallelIndexTuningConfig#targetPartitionSize} *
  • *
  • - * If the number of rows added to {@link BaseAppenderatorDriver} so far exceeds {@link SinglePhaseParallelIndexTuningConfig#maxTotalRows} + * If the number of rows added to {@link BaseAppenderatorDriver} so far exceeds {@link ParallelIndexTuningConfig#maxTotalRows} *
  • *
* @@ -290,7 +291,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null } final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); - final SinglePhaseParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); + final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final long pushTimeout = tuningConfig.getPushTimeout(); final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); @@ -398,7 +399,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null private static boolean exceedMaxRowsInSegment( int numRowsInSegment, - SinglePhaseParallelIndexTuningConfig indexTuningConfig + ParallelIndexTuningConfig indexTuningConfig ) { // maxRowsInSegment should be null if numShards is set in indexTuningConfig @@ -408,7 +409,7 @@ private static boolean exceedMaxRowsInSegment( private static boolean exceedMaxRowsInAppenderator( long numRowsInAppenderator, - SinglePhaseParallelIndexTuningConfig indexTuningConfig + ParallelIndexTuningConfig indexTuningConfig ) { // maxRowsInAppenderator should be null if numShards is set in indexTuningConfig @@ -420,7 +421,7 @@ private static Appenderator newAppenderator( FireDepartmentMetrics metrics, TaskToolbox toolbox, DataSchema dataSchema, - SinglePhaseParallelIndexTuningConfig tuningConfig + ParallelIndexTuningConfig tuningConfig ) { return Appenderators.createOffline( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java similarity index 77% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java index 21129b068a35..156c776a897d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java @@ -25,16 +25,16 @@ import java.util.Map; -class SinglePhaseParallelIndexSubTaskSpec extends SubTaskSpec +class ParallelIndexSubTaskSpec extends SubTaskSpec { - private final SinglePhaseParallelIndexIngestionSpec ingestionSpec; + private final ParallelIndexIngestionSpec ingestionSpec; @JsonCreator - SinglePhaseParallelIndexSubTaskSpec( + ParallelIndexSubTaskSpec( String id, String groupId, String supervisorTaskId, - SinglePhaseParallelIndexIngestionSpec indexIngestionSpec, + ParallelIndexIngestionSpec indexIngestionSpec, Map context, InputSplit inputSplit ) @@ -44,15 +44,15 @@ class SinglePhaseParallelIndexSubTaskSpec extends SubTaskSpec baseFirehoseFactory; + private final IndexingServiceClient indexingServiceClient; + private final ChatHandlerProvider chatHandlerProvider; + private final AuthorizerMapper authorizerMapper; + + private ParallelIndexTaskRunner runner; + + @JsonCreator + public ParallelIndexSupervisorTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("spec") ParallelIndexIngestionSpec ingestionSchema, + @JsonProperty("context") Map context, + @JacksonInject @Nullable IndexingServiceClient indexingServiceClient, // null in overlords + @JacksonInject @Nullable ChatHandlerProvider chatHandlerProvider, // null in overlords + @JacksonInject AuthorizerMapper authorizerMapper + ) + { + super( + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + null, + taskResource, + ingestionSchema.getDataSchema().getDataSource(), + context + ); + + this.ingestionSchema = ingestionSchema; + + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + if (!(firehoseFactory instanceof FiniteFirehoseFactory)) { + throw new IAE("[%s] should implement FiniteFirehoseFactory", firehoseFactory.getClass().getSimpleName()); + } + + this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; + this.indexingServiceClient = indexingServiceClient; + this.chatHandlerProvider = chatHandlerProvider; + this.authorizerMapper = authorizerMapper; + + if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { + log.warn("maxSavedParseExceptions is not supported yet"); + } + if (ingestionSchema.getTuningConfig().getMaxParseExceptions() > 0) { + log.warn("maxParseExceptions is not supported yet"); + } + if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { + log.warn("logParseExceptions is not supported yet"); + } + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); + } + + @Override + public String getType() + { + return TYPE; + } + + @JsonProperty("spec") + public ParallelIndexIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @VisibleForTesting + @Nullable + ParallelIndexTaskRunner getRunner() + { + return runner; + } + + @VisibleForTesting + AuthorizerMapper getAuthorizerMapper() + { + return authorizerMapper; + } + + @VisibleForTesting + ParallelIndexTaskRunner createRunner() + { + if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { + throw new UnsupportedOperationException("Perfect roll-up is not supported yet"); + } else { + runner = new SinglePhaseParallelIndexTaskRunner( + getId(), + getGroupId(), + ingestionSchema, + getContext(), + indexingServiceClient, + chatHandlerProvider, + authorizerMapper + ); + } + return runner; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + final Optional> intervals = ingestionSchema.getDataSchema() + .getGranularitySpec() + .bucketIntervals(); + + return !intervals.isPresent() || isReady(taskActionClient, intervals.get()); + } + + static boolean isReady(TaskActionClient actionClient, SortedSet intervals) throws IOException + { + final List locks = getTaskLocks(actionClient); + if (locks.size() == 0) { + try { + Tasks.tryAcquireExclusiveLocks(actionClient, intervals); + } + catch (Exception e) { + log.error(e, "Failed to acquire locks for intervals[%s]", intervals); + return false; + } + } + return true; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + if (baseFirehoseFactory.isSplittable()) { + return runParallel(toolbox); + } else { + log.warn( + "firehoseFactory[%s] is not splittable. Running sequentially", + baseFirehoseFactory.getClass().getSimpleName() + ); + return runSequential(toolbox); + } + } + + private TaskStatus runParallel(TaskToolbox toolbox) throws Exception + { + createRunner(); + return TaskStatus.fromCode(getId(), runner.run(toolbox)); + } + + private TaskStatus runSequential(TaskToolbox toolbox) throws Exception + { + return new IndexTask( + getId(), + getGroupId(), + getTaskResource(), + getDataSource(), + new IndexIngestionSpec( + getIngestionSchema().getDataSchema(), + getIngestionSchema().getIOConfig(), + convertToIndexTuningConfig(getIngestionSchema().getTuningConfig()) + ), + getContext(), + authorizerMapper, + chatHandlerProvider + ).run(toolbox); + } + + private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningConfig tuningConfig) + { + return new IndexTuningConfig( + tuningConfig.getTargetPartitionSize(), + tuningConfig.getMaxRowsInMemory(), + tuningConfig.getMaxBytesInMemory(), + tuningConfig.getMaxTotalRows(), + null, + tuningConfig.getNumShards(), + tuningConfig.getIndexSpec(), + tuningConfig.getMaxPendingPersists(), + true, + tuningConfig.isForceExtendableShardSpecs(), + false, + tuningConfig.isReportParseExceptions(), + null, + tuningConfig.getPushTimeout(), + tuningConfig.getSegmentWriteOutMediumFactory(), + tuningConfig.isLogParseExceptions(), + tuningConfig.getMaxParseExceptions(), + tuningConfig.getMaxSavedParseExceptions() + ); + } + + static class Status + { + private final int running; + private final int succeeded; + private final int failed; + private final int complete; + private final int total; + private final int expectedSucceeded; + + static Status empty() + { + return new Status(0, 0, 0, 0, 0, 0); + } + + @JsonCreator + Status( + @JsonProperty("running") int running, + @JsonProperty("succeeded") int succeeded, + @JsonProperty("failed") int failed, + @JsonProperty("complete") int complete, + @JsonProperty("total") int total, + @JsonProperty("expectedSucceeded") int expectedSucceeded + ) + { + this.running = running; + this.succeeded = succeeded; + this.failed = failed; + this.complete = complete; + this.total = total; + this.expectedSucceeded = expectedSucceeded; + } + + @JsonProperty + public int getRunning() + { + return running; + } + + @JsonProperty + public int getSucceeded() + { + return succeeded; + } + + @JsonProperty + public int getFailed() + { + return failed; + } + + @JsonProperty + public int getComplete() + { + return complete; + } + + @JsonProperty + public int getTotal() + { + return total; + } + + @JsonProperty + public int getExpectedSucceeded() + { + return expectedSucceeded; + } + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java similarity index 95% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java index fb8e109aaee3..23a4ce71385d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java @@ -32,11 +32,11 @@ import java.io.IOException; import java.util.List; -public class SinglePhaseParallelIndexTaskClient extends IndexTaskClient +public class ParallelIndexTaskClient extends IndexTaskClient { private final String subtaskId; - public SinglePhaseParallelIndexTaskClient( + public ParallelIndexTaskClient( HttpClient httpClient, ObjectMapper objectMapper, TaskInfoProvider taskInfoProvider, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java similarity index 86% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClientFactory.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java index be6679988710..8ce2a3fd2370 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java @@ -28,14 +28,13 @@ import io.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; -public class SinglePhaseParallelIndexTaskClientFactory - implements IndexTaskClientFactory +public class ParallelIndexTaskClientFactory implements IndexTaskClientFactory { private final HttpClient httpClient; private final ObjectMapper mapper; @Inject - public SinglePhaseParallelIndexTaskClientFactory( + public ParallelIndexTaskClientFactory( @EscalatedGlobal HttpClient httpClient, @Smile ObjectMapper mapper ) @@ -45,7 +44,7 @@ public SinglePhaseParallelIndexTaskClientFactory( } @Override - public SinglePhaseParallelIndexTaskClient build( + public ParallelIndexTaskClient build( TaskInfoProvider taskInfoProvider, String callerId, int numThreads, @@ -54,7 +53,7 @@ public SinglePhaseParallelIndexTaskClient build( ) { Preconditions.checkState(numThreads == 1, "expect numThreads to be 1"); - return new SinglePhaseParallelIndexTaskClient( + return new ParallelIndexTaskClient( httpClient, mapper, taskInfoProvider, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java new file mode 100644 index 000000000000..90d02c2122f6 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java @@ -0,0 +1,32 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task; + +import io.druid.indexer.TaskState; +import io.druid.indexing.common.TaskToolbox; + +/** + * ParallelIndexTaskRunner is the actual task runner of {@link ParallelIndexSupervisorTask}. There is currently a single + * implementation, i.e. {@link SinglePhaseParallelIndexTaskRunner} which supports only best-effort roll-up. We can add + * more implementations in the future. + */ +public interface ParallelIndexTaskRunner +{ + TaskState run(TaskToolbox toolbox) throws Exception; +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java similarity index 86% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java index 47179d66a650..0d027e39f39a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java @@ -31,8 +31,8 @@ import javax.annotation.Nullable; import java.util.Objects; -@JsonTypeName("index_single_phase_parallel") -public class SinglePhaseParallelIndexTuningConfig extends IndexTuningConfig +@JsonTypeName("index_parallel") +public class ParallelIndexTuningConfig extends IndexTuningConfig { private static final int DEFAULT_MAX_NUM_BATCH_TASKS = Integer.MAX_VALUE; // unlimited private static final int DEFAULT_MAX_RETRY = 3; @@ -41,16 +41,17 @@ public class SinglePhaseParallelIndexTuningConfig extends IndexTuningConfig private static final Duration DEFAULT_CHAT_HANDLER_TIMEOUT = new Period("PT10S").toStandardDuration(); private static final int DEFAULT_CHAT_HANDLER_NUM_RETRIES = 5; - private final int maxNumBatchTasks; + private final int maxNumSubTasks; private final int maxRetry; private final long taskStatusCheckPeriodMs; private final Duration chatHandlerTimeout; private final int chatHandlerNumRetries; - public static SinglePhaseParallelIndexTuningConfig defaultConfig() + public static ParallelIndexTuningConfig defaultConfig() { - return new SinglePhaseParallelIndexTuningConfig( + return new ParallelIndexTuningConfig( + null, null, null, null, @@ -74,7 +75,7 @@ public static SinglePhaseParallelIndexTuningConfig defaultConfig() } @JsonCreator - public SinglePhaseParallelIndexTuningConfig( + public ParallelIndexTuningConfig( @JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @@ -83,10 +84,11 @@ public SinglePhaseParallelIndexTuningConfig( @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs, + @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, @JsonProperty("pushTimeout") @Nullable Long pushTimeout, @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, - @JsonProperty("maxNumBatchTasks") @Nullable Integer maxNumBatchTasks, + @JsonProperty("maxNumSubTasks") @Nullable Integer maxNumSubTasks, @JsonProperty("maxRetry") @Nullable Integer maxRetry, @JsonProperty("taskStatusCheckPeriodMs") @Nullable Integer taskStatusCheckPeriodMs, @JsonProperty("chatHandlerTimeout") @Nullable Duration chatHandlerTimeout, @@ -107,7 +109,7 @@ public SinglePhaseParallelIndexTuningConfig( maxPendingPersists, null, forceExtendableShardSpecs, - false, // SinglePhaseParallelIndexSupervisorTask can't be used for guaranteed rollup + forceGuaranteedRollup, reportParseExceptions, null, pushTimeout, @@ -117,7 +119,7 @@ public SinglePhaseParallelIndexTuningConfig( maxSavedParseExceptions ); - this.maxNumBatchTasks = maxNumBatchTasks == null ? DEFAULT_MAX_NUM_BATCH_TASKS : maxNumBatchTasks; + this.maxNumSubTasks = maxNumSubTasks == null ? DEFAULT_MAX_NUM_BATCH_TASKS : maxNumSubTasks; this.maxRetry = maxRetry == null ? DEFAULT_MAX_RETRY : maxRetry; this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs == null ? DEFAULT_TASK_STATUS_CHECK_PERIOD_MS : @@ -128,9 +130,9 @@ public SinglePhaseParallelIndexTuningConfig( } @JsonProperty - public int getMaxNumBatchTasks() + public int getMaxNumSubTasks() { - return maxNumBatchTasks; + return maxNumSubTasks; } @JsonProperty @@ -169,8 +171,8 @@ public boolean equals(Object o) if (!super.equals(o)) { return false; } - SinglePhaseParallelIndexTuningConfig that = (SinglePhaseParallelIndexTuningConfig) o; - return maxNumBatchTasks == that.maxNumBatchTasks && + ParallelIndexTuningConfig that = (ParallelIndexTuningConfig) o; + return maxNumSubTasks == that.maxNumSubTasks && maxRetry == that.maxRetry && taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs && chatHandlerNumRetries == that.chatHandlerNumRetries && @@ -183,7 +185,7 @@ public int hashCode() return Objects.hash( super.hashCode(), - maxNumBatchTasks, + maxNumSubTasks, maxRetry, taskStatusCheckPeriodMs, chatHandlerTimeout, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java similarity index 66% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java index 1124a85a16c3..0dc3119b9c17 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package io.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JacksonInject; @@ -24,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -36,17 +34,12 @@ import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; -import io.druid.indexing.common.TaskLock; -import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; -import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status; import io.druid.indexing.common.task.TaskMonitor.MonitorEntry; import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; import io.druid.indexing.common.task.TaskMonitor.TaskHistory; -import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.appenderator.SegmentIdentifier; @@ -58,7 +51,6 @@ import io.druid.server.security.Action; import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; -import org.joda.time.Interval; import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; @@ -79,7 +71,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -89,145 +80,68 @@ import java.util.stream.Stream; /** - * SinglePhaseParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing. This is - * applicable if the input {@link FiniteFirehoseFactory} is splittable. While this task is running, it can submit - * multiple child tasks to overlords. This task succeeds only when all its child tasks succeed; otherwise it fails. + * An implementation of {@link ParallelIndexTaskRunner} to support best-effort roll-up. This runner can submit and + * monitor multiple {@link ParallelIndexSubTask}s. * * As its name indicates, distributed indexing is done in a single phase, i.e., without shuffling intermediate data. As * a result, this task can't be used for perfect rollup. */ -public class SinglePhaseParallelIndexSupervisorTask extends AbstractTask implements ChatHandler +public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunner, ChatHandler { - private static final Logger log = new Logger(SinglePhaseParallelIndexSupervisorTask.class); - private static final String TYPE = "index_single_phase_parallel"; + private static final Logger log = new Logger(SinglePhaseParallelIndexTaskRunner.class); - private final SinglePhaseParallelIndexIngestionSpec ingestionSchema; + private final String taskId; + private final String groupId; + private final ParallelIndexIngestionSpec ingestionSchema; + private final Map context; private final FiniteFirehoseFactory baseFirehoseFactory; private final int maxNumTasks; private final IndexingServiceClient indexingServiceClient; private final ChatHandlerProvider chatHandlerProvider; private final AuthorizerMapper authorizerMapper; - private final BlockingQueue> taskCompleteEvents = + private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); // subTaskId -> report private final ConcurrentMap segmentsMap = new ConcurrentHashMap<>(); private volatile boolean stopped; - private volatile TaskMonitor taskMonitor; + private volatile TaskMonitor taskMonitor; private int nextSpecId = 0; - @JsonCreator - public SinglePhaseParallelIndexSupervisorTask( - @JsonProperty("id") String id, - @JsonProperty("resource") TaskResource taskResource, - @JsonProperty("spec") SinglePhaseParallelIndexIngestionSpec ingestionSchema, + SinglePhaseParallelIndexTaskRunner( + @JsonProperty("id") String taskId, + String groupId, + @JsonProperty("spec") ParallelIndexIngestionSpec ingestionSchema, @JsonProperty("context") Map context, - @JacksonInject @Nullable IndexingServiceClient indexingServiceClient, // null in overlords - @JacksonInject @Nullable ChatHandlerProvider chatHandlerProvider, // null in overlords + @JacksonInject IndexingServiceClient indexingServiceClient, + @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthorizerMapper authorizerMapper ) { - super( - getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), - null, - taskResource, - ingestionSchema.getDataSchema().getDataSource(), - context - ); - + this.taskId = taskId; + this.groupId = groupId; this.ingestionSchema = ingestionSchema; - - final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); - if (!(firehoseFactory instanceof FiniteFirehoseFactory)) { - throw new IAE("[%s] should implement FiniteFirehoseFactory", firehoseFactory.getClass().getSimpleName()); - } - - this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; - this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumBatchTasks(); - this.indexingServiceClient = indexingServiceClient; - this.chatHandlerProvider = chatHandlerProvider; + this.context = context; + this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory(); + this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumSubTasks(); + this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"); + this.chatHandlerProvider = Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider"); this.authorizerMapper = authorizerMapper; - - if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) { - log.warn("maxSavedParseExceptions is not supported yet"); - } - if (ingestionSchema.getTuningConfig().getMaxParseExceptions() > 0) { - log.warn("maxParseExceptions is not supported yet"); - } - if (ingestionSchema.getTuningConfig().isLogParseExceptions()) { - log.warn("logParseExceptions is not supported yet"); - } - } - - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); - } - - @Override - public String getType() - { - return TYPE; - } - - @JsonProperty("spec") - public SinglePhaseParallelIndexIngestionSpec getIngestionSchema() - { - return ingestionSchema; } @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception - { - final Optional> intervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals(); - - return !intervals.isPresent() || isReady(taskActionClient, intervals.get()); - } - - static boolean isReady(TaskActionClient actionClient, SortedSet intervals) throws IOException - { - final List locks = getTaskLocks(actionClient); - if (locks.size() == 0) { - try { - Tasks.tryAcquireExclusiveLocks(actionClient, intervals); - } - catch (Exception e) { - log.error(e, "Failed to acquire locks for intervals[%s]", intervals); - return false; - } - } - return true; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - if (baseFirehoseFactory.isSplittable()) { - return runParallel(toolbox); - } else { - log.warn( - "firehoseFactory[%s] is not splittable. Running sequentially", - baseFirehoseFactory.getClass().getSimpleName() - ); - return runSequential(toolbox); - } - } - - private TaskStatus runParallel(TaskToolbox toolbox) throws Exception + public TaskState run(TaskToolbox toolbox) throws Exception { log.info( "Found chat handler of class[%s]", Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName() ); - chatHandlerProvider.register(getId(), this, false); + chatHandlerProvider.register(taskId, this, false); - final Iterator subTaskSpecIterator = subTaskSpecIterator().iterator(); + final Iterator subTaskSpecIterator = subTaskSpecIterator().iterator(); final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs(); taskMonitor = new TaskMonitor<>( @@ -248,7 +162,7 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception log.info("Waiting for subTasks to be completed"); while (!stopped && !Thread.currentThread().isInterrupted()) { - final SubTaskCompleteEvent taskCompleteEvent = taskCompleteEvents.poll( + final SubTaskCompleteEvent taskCompleteEvent = taskCompleteEvents.poll( taskStatusCheckingPeriod, TimeUnit.MILLISECONDS ); @@ -281,9 +195,9 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception final Status monitorStatus = taskMonitor.getStatus(); throw new ISE( "Expected for [%d] tasks to succeed, but we got [%d] succeeded tasks and [%d] failed tasks", - monitorStatus.expectedSucceeded, - monitorStatus.succeeded, - monitorStatus.failed + monitorStatus.getExpectedSucceeded(), + monitorStatus.getSucceeded(), + monitorStatus.getFailed() ); } } @@ -303,8 +217,8 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception if (lastStatus != null) { log.error("Failed because of the failed sub task[%s]", lastStatus.getId()); } else { - final SinglePhaseParallelIndexSubTaskSpec spec = - (SinglePhaseParallelIndexSubTaskSpec) taskCompleteEvent.getSpec(); + final ParallelIndexSubTaskSpec spec = + (ParallelIndexSubTaskSpec) taskCompleteEvent.getSpec(); log.error( "Failed to run sub tasks for inputSplits[%s]", getSplitsIfSplittable(spec.getIngestionSpec().getIOConfig().getFirehoseFactory()) @@ -322,7 +236,7 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception // Cleanup resources taskCompleteEvents.clear(); taskMonitor.stop(); - chatHandlerProvider.unregister(getId()); + chatHandlerProvider.unregister(taskId); if (state != TaskState.SUCCESS) { log.info( @@ -337,25 +251,18 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception } } - return TaskStatus.fromCode(getId(), state); + return state; } - private TaskStatus runSequential(TaskToolbox toolbox) throws Exception + @VisibleForTesting + ParallelIndexIngestionSpec getIngestionSchema() { - return new IndexTask( - getId(), - getGroupId(), - getTaskResource(), - getDataSource(), - new IndexIngestionSpec( - getIngestionSchema().getDataSchema(), - getIngestionSchema().getIOConfig(), - convertToIndexTuningConfig(getIngestionSchema().getTuningConfig()) - ), - getContext(), - authorizerMapper, - chatHandlerProvider - ).run(toolbox); + return ingestionSchema; + } + + private String getDataSource() + { + return ingestionSchema.getDataSchema().getDataSource(); } // Internal API for collecting reports from subTasks @@ -368,7 +275,12 @@ public Response report( @Context final HttpServletRequest req ) { - ChatHandlers.authorizationCheck(req, Action.WRITE, getDataSource(), authorizerMapper); + ChatHandlers.authorizationCheck( + req, + Action.WRITE, + getDataSource(), + authorizerMapper + ); collectReport(report); return Response.status(Response.Status.OK).build(); } @@ -392,30 +304,6 @@ void collectReport(PushedSegmentsReport report) }); } - private static IndexTuningConfig convertToIndexTuningConfig(SinglePhaseParallelIndexTuningConfig tuningConfig) - { - return new IndexTuningConfig( - tuningConfig.getTargetPartitionSize(), - tuningConfig.getMaxRowsInMemory(), - tuningConfig.getMaxBytesInMemory(), - tuningConfig.getMaxTotalRows(), - null, - tuningConfig.getNumShards(), - tuningConfig.getIndexSpec(), - tuningConfig.getMaxPendingPersists(), - true, - tuningConfig.isForceExtendableShardSpecs(), - false, - tuningConfig.isReportParseExceptions(), - null, - tuningConfig.getPushTimeout(), - tuningConfig.getSegmentWriteOutMediumFactory(), - tuningConfig.isLogParseExceptions(), - tuningConfig.getMaxParseExceptions(), - tuningConfig.getMaxSavedParseExceptions() - ); - } - private void publish(TaskToolbox toolbox) throws IOException { final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { @@ -450,18 +338,18 @@ private void publish(TaskToolbox toolbox) throws IOException } private void submitNewTask( - TaskMonitor taskMonitor, - SinglePhaseParallelIndexSubTaskSpec spec + TaskMonitor taskMonitor, + ParallelIndexSubTaskSpec spec ) { log.info("Submit a new task for spec[%s] and inputSplit[%s]", spec.getId(), spec.getInputSplit()); - final ListenableFuture> future = taskMonitor.submit(spec); + final ListenableFuture> future = taskMonitor.submit(spec); Futures.addCallback( future, - new FutureCallback>() + new FutureCallback>() { @Override - public void onSuccess(SubTaskCompleteEvent completeEvent) + public void onSuccess(SubTaskCompleteEvent completeEvent) { // this callback is called if a task completed wheter it succeeded or not. taskCompleteEvents.offer(completeEvent); @@ -485,27 +373,27 @@ int getAndIncrementNextSpecId() } @VisibleForTesting - Stream subTaskSpecIterator() throws IOException + Stream subTaskSpecIterator() throws IOException { return baseFirehoseFactory.getSplits().map(this::newTaskSpec); } @VisibleForTesting - SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) { - return new SinglePhaseParallelIndexSubTaskSpec( - getId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), - getId(), - new SinglePhaseParallelIndexIngestionSpec( + return new ParallelIndexSubTaskSpec( + taskId + "_" + getAndIncrementNextSpecId(), + groupId, + taskId, + new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( baseFirehoseFactory.withSplit(split), ingestionSchema.getIOConfig().isAppendToExisting() ), ingestionSchema.getTuningConfig() ), - getContext(), + context, split ); } @@ -557,12 +445,12 @@ public Response getSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (taskMonitor != null) { - final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); - final List> completeSubTaskSpecs = taskMonitor + final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); + final List> completeSubTaskSpecs = taskMonitor .getCompleteSubTaskSpecs(); // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and // completeSubTaskSpecs. - final Map> subTaskSpecMap = new HashMap<>( + final Map> subTaskSpecMap = new HashMap<>( runningSubTaskSpecs.size() + completeSubTaskSpecs.size() ); runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); @@ -579,9 +467,9 @@ public Response getSubTaskSpecs(@Context final HttpServletRequest req) public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final List> runningSubTaskSpecs = taskMonitor == null ? - Collections.emptyList() : - taskMonitor.getRunningSubTaskSpecs(); + final List> runningSubTaskSpecs = taskMonitor == null ? + Collections.emptyList() : + taskMonitor.getRunningSubTaskSpecs(); return Response.ok(runningSubTaskSpecs).build(); } @@ -591,7 +479,7 @@ public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final List> completeSubTaskSpecs = + final List> completeSubTaskSpecs = taskMonitor == null ? Collections.emptyList() : taskMonitor.getCompleteSubTaskSpecs(); @@ -608,8 +496,8 @@ public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpSe // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after // adding them to taskHistory. final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); - final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); - final SubTaskSpec subTaskSpec; + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); + final SubTaskSpec subTaskSpec; if (monitorEntry != null) { subTaskSpec = monitorEntry.getSpec(); @@ -643,20 +531,20 @@ public Response getSubTaskState(@PathParam("id") String id, @Context final HttpS // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after // adding them to taskHistory. final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); - final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); final SubTaskStateResponse subTaskStateResponse; if (monitorEntry != null) { subTaskStateResponse = new SubTaskStateResponse( - (SinglePhaseParallelIndexSubTaskSpec) monitorEntry.getSpec(), + (ParallelIndexSubTaskSpec) monitorEntry.getSpec(), monitorEntry.getRunningStatus(), monitorEntry.getTaskHistory() ); } else { if (taskHistory != null && !taskHistory.isEmpty()) { subTaskStateResponse = new SubTaskStateResponse( - (SinglePhaseParallelIndexSubTaskSpec) taskHistory.getSpec(), + (ParallelIndexSubTaskSpec) taskHistory.getSpec(), null, taskHistory.getAttemptHistory() ); @@ -685,7 +573,7 @@ public Response getCompleteSubTaskSpecAttemptHistory( if (taskMonitor == null) { return Response.status(Response.Status.NOT_FOUND).build(); } else { - final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); if (taskHistory == null) { return Response.status(Response.Status.NOT_FOUND).build(); @@ -695,85 +583,16 @@ public Response getCompleteSubTaskSpecAttemptHistory( } } - static class Status - { - private final int running; - private final int succeeded; - private final int failed; - private final int complete; - private final int total; - private final int expectedSucceeded; - - static Status empty() - { - return new Status(0, 0, 0, 0, 0, 0); - } - - @JsonCreator - Status( - @JsonProperty("running") int running, - @JsonProperty("succeeded") int succeeded, - @JsonProperty("failed") int failed, - @JsonProperty("complete") int complete, - @JsonProperty("total") int total, - @JsonProperty("expectedSucceeded") int expectedSucceeded - ) - { - this.running = running; - this.succeeded = succeeded; - this.failed = failed; - this.complete = complete; - this.total = total; - this.expectedSucceeded = expectedSucceeded; - } - - @JsonProperty - public int getRunning() - { - return running; - } - - @JsonProperty - public int getSucceeded() - { - return succeeded; - } - - @JsonProperty - public int getFailed() - { - return failed; - } - - @JsonProperty - public int getComplete() - { - return complete; - } - - @JsonProperty - public int getTotal() - { - return total; - } - - @JsonProperty - public int getExpectedSucceeded() - { - return expectedSucceeded; - } - } - static class SubTaskStateResponse { - private final SinglePhaseParallelIndexSubTaskSpec spec; + private final ParallelIndexSubTaskSpec spec; @Nullable private final TaskStatusPlus currentStatus; private final List taskHistory; @JsonCreator public SubTaskStateResponse( - @JsonProperty("spec") SinglePhaseParallelIndexSubTaskSpec spec, + @JsonProperty("spec") ParallelIndexSubTaskSpec spec, @JsonProperty("currentStatus") @Nullable TaskStatusPlus currentStatus, @JsonProperty("taskHistory") List taskHistory ) @@ -784,7 +603,7 @@ public SubTaskStateResponse( } @JsonProperty - public SinglePhaseParallelIndexSubTaskSpec getSpec() + public ParallelIndexSubTaskSpec getSpec() { return spec; } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index c4f9fad805da..4fadaa53527e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -51,8 +51,8 @@ @JsonSubTypes.Type(name = "archive", value = ArchiveTask.class), @JsonSubTypes.Type(name = "restore", value = RestoreTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), - @JsonSubTypes.Type(name = "index_single_phase_parallel", value = SinglePhaseParallelIndexSupervisorTask.class), - @JsonSubTypes.Type(name = "index_single_phase_sub", value = SinglePhaseParallelIndexSubTask.class), + @JsonSubTypes.Type(name = ParallelIndexSupervisorTask.TYPE, value = ParallelIndexSupervisorTask.class), + @JsonSubTypes.Type(name = ParallelIndexSubTask.TYPE, value = ParallelIndexSubTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment", value = HadoopConverterTask.class), @JsonSubTypes.Type(name = "hadoop_convert_segment_sub", value = HadoopConverterTask.ConverterSubTask.class), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index d6d5176ad372..639580fd7340 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -28,7 +28,7 @@ import io.druid.client.indexing.TaskStatusResponse; import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; -import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.Status; +import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java similarity index 78% rename from indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java index b3ec64bb0330..4d5ffd649c21 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractSinglePhaseParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java @@ -45,6 +45,7 @@ import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusherConfig; +import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.Authorizer; @@ -54,9 +55,11 @@ import org.junit.Rule; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -65,7 +68,7 @@ import java.util.concurrent.Future; import java.util.stream.Stream; -public class AbstractSinglePhaseParallelIndexSupervisorTaskTest extends IngestionTestBase +public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase { static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec( new TimestampSpec( @@ -96,13 +99,13 @@ class LocalIndexingServiceClient extends NoopIndexingServiceClient { private final ConcurrentMap> tasks = new ConcurrentHashMap<>(); private final ListeningExecutorService service = MoreExecutors.listeningDecorator( - Execs.multiThreaded(5, "parallel-index-single-phase-supervisor-task-test-%d") + Execs.multiThreaded(5, "parallel-index-supervisor-task-test-%d") ); @Override public String runTask(Object taskObject) { - final SinglePhaseParallelIndexSubTask subTask = (SinglePhaseParallelIndexSubTask) taskObject; + final ParallelIndexSubTask subTask = (ParallelIndexSubTask) taskObject; tasks.put(subTask.getId(), service.submit(() -> { try { final TaskToolbox toolbox = createTaskToolbox(subTask); @@ -131,7 +134,7 @@ public TaskStatusResponse getTaskStatus(String taskId) taskId, new TaskStatusPlus( taskId, - "index_single_phase_sub", + "index_sub", DateTimes.EPOCH, DateTimes.EPOCH, taskStatus.getStatusCode(), @@ -146,7 +149,7 @@ public TaskStatusResponse getTaskStatus(String taskId) taskId, new TaskStatusPlus( taskId, - "index_single_phase_sub", + "index_sub", DateTimes.EPOCH, DateTimes.EPOCH, TaskState.RUNNING, @@ -166,7 +169,7 @@ public TaskStatusResponse getTaskStatus(String taskId) taskId, new TaskStatusPlus( taskId, - "index_single_phase_sub", + "index_sub", DateTimes.EPOCH, DateTimes.EPOCH, TaskState.FAILED, @@ -252,12 +255,12 @@ public void killAll() ); } - static class TestSinglePhaseParallelIndexSupervisorTask extends SinglePhaseParallelIndexSupervisorTask + static class TestParallelIndexSupervisorTask extends ParallelIndexSupervisorTask { - TestSinglePhaseParallelIndexSupervisorTask( + TestParallelIndexSupervisorTask( String id, TaskResource taskResource, - SinglePhaseParallelIndexIngestionSpec ingestionSchema, + ParallelIndexIngestionSpec ingestionSchema, Map context, IndexingServiceClient indexingServiceClient ) @@ -281,7 +284,40 @@ public Authorizer getAuthorizer(String name) } @Override - Stream subTaskSpecIterator() throws IOException + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + return TaskStatus.fromCode( + getId(), + new TestParallelIndexTaskRunner( + getId(), + getGroupId(), + getIngestionSchema(), + getContext(), + new NoopIndexingServiceClient(), + new NoopChatHandlerProvider(), + new AuthorizerMapper(Collections.emptyMap()) + ).run(toolbox) + ); + } + } + + static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRunner + { + TestParallelIndexTaskRunner( + String taskId, + String groupId, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + @Nullable IndexingServiceClient indexingServiceClient, + @Nullable ChatHandlerProvider chatHandlerProvider, + AuthorizerMapper authorizerMapper + ) + { + super(taskId, groupId, ingestionSchema, context, indexingServiceClient, chatHandlerProvider, authorizerMapper); + } + + @Override + Stream subTaskSpecIterator() throws IOException { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() @@ -299,18 +335,17 @@ Stream subTaskSpecIterator() throws IOExcep } } - static class LocalSinglePhaseParallelIndexTaskClientFactory - implements IndexTaskClientFactory + static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory { - private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + private final SinglePhaseParallelIndexTaskRunner runner; - LocalSinglePhaseParallelIndexTaskClientFactory(SinglePhaseParallelIndexSupervisorTask supervisorTask) + LocalParallelIndexTaskClientFactory(SinglePhaseParallelIndexTaskRunner runner) { - this.supervisorTask = supervisorTask; + this.runner = runner; } @Override - public SinglePhaseParallelIndexTaskClient build( + public ParallelIndexTaskClient build( TaskInfoProvider taskInfoProvider, String callerId, int numThreads, @@ -318,24 +353,24 @@ public SinglePhaseParallelIndexTaskClient build( long numRetries ) { - return new LocalSinglePhaseParallelIndexTaskClient(callerId, supervisorTask); + return new LocalParallelIndexTaskClient(callerId, runner); } } - static class LocalSinglePhaseParallelIndexTaskClient extends SinglePhaseParallelIndexTaskClient + static class LocalParallelIndexTaskClient extends ParallelIndexTaskClient { - private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + private final SinglePhaseParallelIndexTaskRunner runner; - public LocalSinglePhaseParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexSupervisorTask supervisorTask) + public LocalParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexTaskRunner runner) { super(null, null, null, null, callerId, 0); - this.supervisorTask = supervisorTask; + this.runner = runner; } @Override public void report(String supervisorTaskId, List pushedSegments) { - supervisorTask.collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); + runner.collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java similarity index 80% rename from indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java index 15a7f6a4fc22..31d127826c47 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java @@ -33,8 +33,8 @@ import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.Status; -import io.druid.indexing.common.task.SinglePhaseParallelIndexSupervisorTask.SubTaskStateResponse; +import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status; +import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskRunner.SubTaskStateResponse; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; @@ -44,8 +44,11 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.realtime.firehose.ChatHandlerProvider; +import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationResult; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; @@ -55,6 +58,7 @@ import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; import java.io.IOException; @@ -76,13 +80,12 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -public class SinglePhaseParallelIndexSupervisorTaskResourceTest - extends AbstractSinglePhaseParallelIndexSupervisorTaskTest +public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelIndexSupervisorTaskTest { private static final int NUM_SUB_TASKS = 10; // specId -> spec - private final ConcurrentMap subTaskSpecs = new ConcurrentHashMap<>(); + private final ConcurrentMap subTaskSpecs = new ConcurrentHashMap<>(); // specId -> taskStatusPlus private final ConcurrentMap runningSpecs = new ConcurrentHashMap<>(); @@ -91,7 +94,7 @@ public class SinglePhaseParallelIndexSupervisorTaskResourceTest private final ConcurrentMap> taskHistories = new ConcurrentHashMap<>(); // taskId -> subTaskSpec - private final ConcurrentMap taskIdToSpec = new ConcurrentHashMap<>(); + private final ConcurrentMap taskIdToSpec = new ConcurrentHashMap<>(); // taskId -> task private final CopyOnWriteArrayList runningTasks = new CopyOnWriteArrayList<>(); @@ -99,11 +102,12 @@ public class SinglePhaseParallelIndexSupervisorTaskResourceTest private ExecutorService service; private TestSupervisorTask task; + private SinglePhaseParallelIndexTaskRunner runner; @Before public void setup() throws IOException { - service = Execs.singleThreaded("single-phase-parallel-index-supervisor-task-resource-test-%d"); + service = Execs.singleThreaded("parallel-index-supervisor-task-resource-test-%d"); indexingServiceClient = new LocalIndexingServiceClient(); localDeepStorage = temporaryFolder.newFolder("localStorage"); } @@ -121,7 +125,7 @@ public void testAPIs() throws Exception { task = newTask( Intervals.of("2017/2018"), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( new TestFirehose(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())), false ) @@ -132,14 +136,18 @@ public void testAPIs() throws Exception prepareTaskForLocking(task); Assert.assertTrue(task.isReady(actionClient)); final Future supervisorTaskFuture = service.submit(() -> task.run(toolbox)); + Thread.sleep(1000); + + runner = (SinglePhaseParallelIndexTaskRunner) task.getRunner(); + Assert.assertNotNull("runner is null", runner); // test isRunningInParallel - Response response = task.isRunningInParallel(newRequest()); + Response response = runner.isRunningInParallel(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals("parallel", response.getEntity()); // test expectedNumSucceededTasks - response = task.getStatus(newRequest()); + response = runner.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(NUM_SUB_TASKS, ((Status) response.getEntity()).getExpectedSucceeded()); @@ -152,7 +160,6 @@ public void testAPIs() throws Exception int succeededTasks = 0; int failedTasks = 0; checkState( - task, succeededTasks, failedTasks, buildStateMap() @@ -169,7 +176,6 @@ public void testAPIs() throws Exception } checkState( - task, succeededTasks, failedTasks, buildStateMap() @@ -187,7 +193,6 @@ public void testAPIs() throws Exception } checkState( - task, succeededTasks, failedTasks, buildStateMap() @@ -204,7 +209,6 @@ public void testAPIs() throws Exception } checkState( - task, succeededTasks, failedTasks, buildStateMap() @@ -227,7 +231,6 @@ public void testAPIs() throws Exception } checkState( - task, succeededTasks, failedTasks, buildStateMap() @@ -246,7 +249,7 @@ public void testAPIs() throws Exception @SuppressWarnings({"ConstantConditions"}) private int getNumSubTasks(Function func) { - final Response response = task.getStatus(newRequest()); + final Response response = runner.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); return func.apply((Status) response.getEntity()); } @@ -266,16 +269,15 @@ private Map buildStateMap() } /** - * Test all endpoints of {@link SinglePhaseParallelIndexSupervisorTask}. + * Test all endpoints of {@link ParallelIndexSupervisorTask}. */ private void checkState( - TestSupervisorTask task, int expectedSucceededTasks, int expectedFailedTask, Map expectedSubTaskStateResponses // subTaskSpecId -> response ) { - Response response = task.getStatus(newRequest()); + Response response = runner.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); final Status monitorStatus = (Status) response.getEntity(); @@ -295,7 +297,7 @@ private void checkState( Assert.assertEquals(runningTasks.size() + expectedSucceededTasks + expectedFailedTask, monitorStatus.getTotal()); // runningSubTasks - response = task.getRunningTasks(newRequest()); + response = runner.getRunningTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals( runningTasks.stream().map(AbstractTask::getId).collect(Collectors.toSet()), @@ -303,47 +305,47 @@ private void checkState( ); // subTaskSpecs - response = task.getSubTaskSpecs(newRequest()); + response = runner.getSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); - List> actualSubTaskSpecMap = - (List>) response.getEntity(); + List> actualSubTaskSpecMap = + (List>) response.getEntity(); Assert.assertEquals( subTaskSpecs.keySet(), actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) ); // runningSubTaskSpecs - response = task.getRunningSubTaskSpecs(newRequest()); + response = runner.getRunningSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); actualSubTaskSpecMap = - (List>) response.getEntity(); + (List>) response.getEntity(); Assert.assertEquals( runningSpecs.keySet(), actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) ); // completeSubTaskSpecs - final List> completeSubTaskSpecs = expectedSubTaskStateResponses + final List> completeSubTaskSpecs = expectedSubTaskStateResponses .entrySet() .stream() .filter(entry -> !runningSpecs.containsKey(entry.getKey())) .map(entry -> entry.getValue().getSpec()) .collect(Collectors.toList()); - response = task.getCompleteSubTaskSpecs(newRequest()); + response = runner.getCompleteSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(completeSubTaskSpecs, response.getEntity()); // subTaskSpec final String subTaskId = runningSpecs.keySet().iterator().next(); - response = task.getSubTaskSpec(subTaskId, newRequest()); + response = runner.getSubTaskSpec(subTaskId, newRequest()); Assert.assertEquals(200, response.getStatus()); - final SubTaskSpec subTaskSpec = - (SubTaskSpec) response.getEntity(); + final SubTaskSpec subTaskSpec = + (SubTaskSpec) response.getEntity(); Assert.assertEquals(subTaskId, subTaskSpec.getId()); // subTaskState - response = task.getSubTaskState(subTaskId, newRequest()); + response = runner.getSubTaskState(subTaskId, newRequest()); Assert.assertEquals(200, response.getStatus()); final SubTaskStateResponse expectedResponse = Preconditions.checkNotNull( expectedSubTaskStateResponses.get(subTaskId), @@ -368,7 +370,7 @@ private void checkState( .findFirst() .orElse(null); if (completeSubTaskSpecId != null) { - response = task.getCompleteSubTaskSpecAttemptHistory(completeSubTaskSpecId, newRequest()); + response = runner.getCompleteSubTaskSpecAttemptHistory(completeSubTaskSpecId, newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals( expectedSubTaskStateResponses.get(completeSubTaskSpecId).getTaskHistory(), @@ -389,11 +391,11 @@ private static HttpServletRequest newRequest() private TestSupervisorTask newTask( Interval interval, - SinglePhaseParallelIndexIOConfig ioConfig + ParallelIndexIOConfig ioConfig ) { // set up ingestion spec - final SinglePhaseParallelIndexIngestionSpec singlePhaseIngestionSpec = new SinglePhaseParallelIndexIngestionSpec( + final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "dataSource", getObjectMapper().convertValue( @@ -415,7 +417,8 @@ private TestSupervisorTask newTask( getObjectMapper() ), ioConfig, - new SinglePhaseParallelIndexTuningConfig( + new ParallelIndexTuningConfig( + null, null, null, null, @@ -442,8 +445,8 @@ private TestSupervisorTask newTask( return new TestSupervisorTask( null, null, - singlePhaseIngestionSpec, - new HashMap<>(), + ingestionSpec, + Collections.emptyMap(), indexingServiceClient ); } @@ -476,12 +479,14 @@ public FiniteFirehoseFactory withSplit(InputSplit } } - private class TestSupervisorTask extends TestSinglePhaseParallelIndexSupervisorTask + private class TestSupervisorTask extends TestParallelIndexSupervisorTask { + private TestRunner runner; + TestSupervisorTask( String id, TaskResource taskResource, - SinglePhaseParallelIndexIngestionSpec ingestionSchema, + ParallelIndexIngestionSpec ingestionSchema, Map context, IndexingServiceClient indexingServiceClient ) @@ -496,24 +501,70 @@ private class TestSupervisorTask extends TestSinglePhaseParallelIndexSupervisorT } @Override - SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + this.runner = new TestRunner( + this, + indexingServiceClient, + new NoopChatHandlerProvider(), + getAuthorizerMapper() + ); + return TaskStatus.fromCode( + getId(), + runner.run(toolbox) + ); + } + + @Override + public ParallelIndexTaskRunner getRunner() + { + return runner; + } + } + + private class TestRunner extends TestParallelIndexTaskRunner + { + private final ParallelIndexSupervisorTask supervisorTask; + + TestRunner( + ParallelIndexSupervisorTask supervisorTask, + @Nullable IndexingServiceClient indexingServiceClient, + @Nullable ChatHandlerProvider chatHandlerProvider, + AuthorizerMapper authorizerMapper + ) + { + super( + supervisorTask.getId(), + supervisorTask.getGroupId(), + supervisorTask.getIngestionSchema(), + supervisorTask.getContext(), + indexingServiceClient, + chatHandlerProvider, + authorizerMapper + ); + this.supervisorTask = supervisorTask; + } + + @Override + ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() .getFirehoseFactory(); final TestSubTaskSpec spec = new TestSubTaskSpec( - getId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), + supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), + supervisorTask.getGroupId(), + supervisorTask, this, - new SinglePhaseParallelIndexIngestionSpec( + new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( baseFirehoseFactory.withSplit(split), getIngestionSchema().getIOConfig().isAppendToExisting() ), getIngestionSchema().getTuningConfig() ), - getContext(), + supervisorTask.getContext(), split ); subTaskSpecs.put(spec.getId(), spec); @@ -521,25 +572,26 @@ SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) } } - private class TestSubTaskSpec extends SinglePhaseParallelIndexSubTaskSpec + private class TestSubTaskSpec extends ParallelIndexSubTaskSpec { - private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + private final SinglePhaseParallelIndexTaskRunner runner; TestSubTaskSpec( String id, String groupId, - SinglePhaseParallelIndexSupervisorTask supervisorTask, - SinglePhaseParallelIndexIngestionSpec ingestionSpec, + ParallelIndexSupervisorTask supervisorTask, + SinglePhaseParallelIndexTaskRunner runner, + ParallelIndexIngestionSpec ingestionSpec, Map context, InputSplit inputSplit ) { super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit); - this.supervisorTask = supervisorTask; + this.runner = runner; } @Override - public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) + public ParallelIndexSubTask newSubTask(int numAttempts) { try { // taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id @@ -554,7 +606,7 @@ public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) numAttempts, getIngestionSpec(), getContext(), - new LocalSinglePhaseParallelIndexTaskClientFactory(supervisorTask) + new LocalParallelIndexTaskClientFactory(runner) ); final TestFirehose firehose = (TestFirehose) getIngestionSpec().getIOConfig().getFirehoseFactory(); final InputSplit split = firehose.getSplits().findFirst().orElse(null); @@ -581,7 +633,7 @@ public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) } } - private class TestSubTask extends SinglePhaseParallelIndexSubTask + private class TestSubTask extends ParallelIndexSubTask { private volatile TaskState state = TaskState.RUNNING; @@ -589,9 +641,9 @@ private class TestSubTask extends SinglePhaseParallelIndexSubTask String groupId, String supervisorTaskId, int numAttempts, - SinglePhaseParallelIndexIngestionSpec ingestionSchema, + ParallelIndexIngestionSpec ingestionSchema, Map context, - IndexTaskClientFactory taskClientFactory + IndexTaskClientFactory taskClientFactory ) { super( @@ -626,7 +678,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .submit(new SurrogateAction<>(getSupervisorTaskId(), new LockListAction())); Preconditions.checkState(locks.size() == 1, "There should be a single lock"); - task.collectReport( + runner.collectReport( new PushedSegmentsReport( getId(), Collections.singletonList( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java similarity index 90% rename from indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskSerdeTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java index c62539e43bb6..bbb266c9d37d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java @@ -52,7 +52,7 @@ import java.util.HashMap; import java.util.Map; -public class SinglePhaseParallelIndexSupervisorTaskSerdeTest +public class ParallelIndexSupervisorTaskSerdeTest { @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -84,7 +84,7 @@ public void testSerde() throws IOException new NamedType(LocalFirehoseFactory.class, "local") ); - final SinglePhaseParallelIndexSupervisorTask task = newTask( + final ParallelIndexSupervisorTask task = newTask( objectMapper, Intervals.of("2018/2019") ); @@ -92,13 +92,13 @@ public void testSerde() throws IOException Assert.assertEquals(task, objectMapper.readValue(json, Task.class)); } - private SinglePhaseParallelIndexSupervisorTask newTask( + private ParallelIndexSupervisorTask newTask( ObjectMapper objectMapper, Interval interval ) { // set up ingestion spec - final SinglePhaseParallelIndexIngestionSpec singlePhaseIngestionSpec = new SinglePhaseParallelIndexIngestionSpec( + final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "dataSource", objectMapper.convertValue( @@ -119,11 +119,12 @@ private SinglePhaseParallelIndexSupervisorTask newTask( null, objectMapper ), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( new LocalFirehoseFactory(new File("tmp"), "test_*", null), false ), - new SinglePhaseParallelIndexTuningConfig( + new ParallelIndexTuningConfig( + null, null, null, null, @@ -147,10 +148,10 @@ private SinglePhaseParallelIndexSupervisorTask newTask( ); // set up test tools - return new SinglePhaseParallelIndexSupervisorTask( + return new ParallelIndexSupervisorTask( "taskId", new TaskResource("group", 1), - singlePhaseIngestionSpec, + ingestionSpec, new HashMap<>(), new NoopIndexingServiceClient(), new NoopChatHandlerProvider(), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java similarity index 69% rename from indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java index 7837cabda88c..9cb025d837ee 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SinglePhaseParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java @@ -24,6 +24,8 @@ import io.druid.data.input.InputSplit; import io.druid.data.input.impl.StringInputRowParser; import io.druid.indexer.TaskState; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; @@ -32,13 +34,17 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; +import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import io.druid.server.security.AuthorizerMapper; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.Writer; @@ -49,7 +55,7 @@ import java.util.Iterator; import java.util.Map; -public class SinglePhaseParallelIndexSupervisorTaskTest extends AbstractSinglePhaseParallelIndexSupervisorTaskTest +public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest { private File inputDir; @@ -87,9 +93,9 @@ public void teardown() @Test public void testIsReady() throws Exception { - final SinglePhaseParallelIndexSupervisorTask task = newTask( + final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false ) @@ -100,11 +106,12 @@ public void testIsReady() throws Exception prepareTaskForLocking(task); Assert.assertTrue(task.isReady(actionClient)); - final Iterator subTaskSpecIterator = task.subTaskSpecIterator().iterator(); + final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.createRunner(); + final Iterator subTaskSpecIterator = runner.subTaskSpecIterator().iterator(); while (subTaskSpecIterator.hasNext()) { - final SinglePhaseParallelIndexSubTaskSpec spec = subTaskSpecIterator.next(); - final SinglePhaseParallelIndexSubTask subTask = new SinglePhaseParallelIndexSubTask( + final ParallelIndexSubTaskSpec spec = subTaskSpecIterator.next(); + final ParallelIndexSubTask subTask = new ParallelIndexSubTask( null, spec.getGroupId(), null, @@ -124,9 +131,9 @@ public void testIsReady() throws Exception @Test public void testWithoutInterval() throws Exception { - final SinglePhaseParallelIndexSupervisorTask task = newTask( + final ParallelIndexSupervisorTask task = newTask( null, - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false ) @@ -142,9 +149,9 @@ public void testWithoutInterval() throws Exception @Test() public void testRunInParallel() throws Exception { - final SinglePhaseParallelIndexSupervisorTask task = newTask( + final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false ) @@ -160,9 +167,9 @@ public void testRunInParallel() throws Exception @Test public void testRunInSequential() throws Exception { - final SinglePhaseParallelIndexSupervisorTask task = newTask( + final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null) { @Override @@ -182,13 +189,13 @@ public boolean isSplittable() Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); } - private SinglePhaseParallelIndexSupervisorTask newTask( + private ParallelIndexSupervisorTask newTask( Interval interval, - SinglePhaseParallelIndexIOConfig ioConfig + ParallelIndexIOConfig ioConfig ) { // set up ingestion spec - final SinglePhaseParallelIndexIngestionSpec singlePhaseIngestionSpec = new SinglePhaseParallelIndexIngestionSpec( + final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "dataSource", getObjectMapper().convertValue( @@ -210,7 +217,8 @@ private SinglePhaseParallelIndexSupervisorTask newTask( getObjectMapper() ), ioConfig, - new SinglePhaseParallelIndexTuningConfig( + new ParallelIndexTuningConfig( + null, null, null, null, @@ -237,18 +245,20 @@ private SinglePhaseParallelIndexSupervisorTask newTask( return new TestSupervisorTask( null, null, - singlePhaseIngestionSpec, + ingestionSpec, new HashMap<>(), indexingServiceClient ); } - private static class TestSupervisorTask extends TestSinglePhaseParallelIndexSupervisorTask + private static class TestSupervisorTask extends TestParallelIndexSupervisorTask { + private final IndexingServiceClient indexingServiceClient; + TestSupervisorTask( String id, TaskResource taskResource, - SinglePhaseParallelIndexIngestionSpec ingestionSchema, + ParallelIndexIngestionSpec ingestionSchema, Map context, IndexingServiceClient indexingServiceClient ) @@ -260,53 +270,94 @@ private static class TestSupervisorTask extends TestSinglePhaseParallelIndexSupe context, indexingServiceClient ); + this.indexingServiceClient = indexingServiceClient; } @Override - SinglePhaseParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + return TaskStatus.fromCode( + getId(), + new TestRunner( + this, + indexingServiceClient, + new NoopChatHandlerProvider(), + new AuthorizerMapper(Collections.emptyMap()) + ).run(toolbox) + ); + } + } + + private static class TestRunner extends TestParallelIndexTaskRunner + { + private final ParallelIndexSupervisorTask supervisorTask; + + TestRunner( + ParallelIndexSupervisorTask supervisorTask, + @Nullable IndexingServiceClient indexingServiceClient, + @Nullable ChatHandlerProvider chatHandlerProvider, + AuthorizerMapper authorizerMapper + ) + { + super( + supervisorTask.getId(), + supervisorTask.getGroupId(), + supervisorTask.getIngestionSchema(), + supervisorTask.getContext(), + indexingServiceClient, + chatHandlerProvider, + authorizerMapper + ); + this.supervisorTask = supervisorTask; + } + + @Override + ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() .getFirehoseFactory(); - return new TestSinglePhaseParallelIndexSubTaskSpec( - getId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), + return new TestParallelIndexSubTaskSpec( + supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), + supervisorTask.getGroupId(), + supervisorTask, this, - new SinglePhaseParallelIndexIngestionSpec( + new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), - new SinglePhaseParallelIndexIOConfig( + new ParallelIndexIOConfig( baseFirehoseFactory.withSplit(split), getIngestionSchema().getIOConfig().isAppendToExisting() ), getIngestionSchema().getTuningConfig() ), - getContext(), + supervisorTask.getContext(), split ); } } - private static class TestSinglePhaseParallelIndexSubTaskSpec extends SinglePhaseParallelIndexSubTaskSpec + private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSpec { - private final SinglePhaseParallelIndexSupervisorTask supervisorTask; + private final SinglePhaseParallelIndexTaskRunner runner; - TestSinglePhaseParallelIndexSubTaskSpec( + TestParallelIndexSubTaskSpec( String id, String groupId, - SinglePhaseParallelIndexSupervisorTask supervisorTask, - SinglePhaseParallelIndexIngestionSpec ingestionSpec, + ParallelIndexSupervisorTask supervisorTask, + SinglePhaseParallelIndexTaskRunner runner, + ParallelIndexIngestionSpec ingestionSpec, Map context, InputSplit inputSplit ) { super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit); - this.supervisorTask = supervisorTask; + this.runner = runner; } @Override - public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) + public ParallelIndexSubTask newSubTask(int numAttempts) { - return new SinglePhaseParallelIndexSubTask( + return new ParallelIndexSubTask( null, getGroupId(), null, @@ -315,7 +366,7 @@ public SinglePhaseParallelIndexSubTask newSubTask(int numAttempts) getIngestionSpec(), getContext(), null, - new LocalSinglePhaseParallelIndexTaskClientFactory(supervisorTask) + new LocalParallelIndexTaskClientFactory(runner) ); } } diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITSinglePhaseParallelIndexTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java similarity index 85% rename from integration-tests/src/test/java/io/druid/tests/indexer/ITSinglePhaseParallelIndexTest.java rename to integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java index 12a231dd88a4..a37716d617b2 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITSinglePhaseParallelIndexTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java @@ -23,11 +23,11 @@ import org.testng.annotations.Test; @Guice(moduleFactory = DruidTestModuleFactory.class) -public class ITSinglePhaseParallelIndexTest extends AbstractITBatchIndexTest +public class ITParallelIndexTest extends AbstractITBatchIndexTest { - private static String INDEX_TASK = "/indexer/wikipedia_parallel_single_phase_index_task.json"; - private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_single_phase_index_queries.json"; - private static String INDEX_DATASOURCE = "wikipedia_parallel_single_phase_index_test"; + private static String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json"; + private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json"; + private static String INDEX_DATASOURCE = "wikipedia_parallel_index_test"; @Test public void testIndexData() throws Exception diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_queries.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_queries.json similarity index 93% rename from integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_queries.json rename to integration-tests/src/test/resources/indexer/wikipedia_parallel_index_queries.json index 3e3138020aed..76ecb5cd1a36 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_queries.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_queries.json @@ -3,7 +3,7 @@ "description": "timeseries, 1 agg, all", "query":{ "queryType" : "timeBoundary", - "dataSource": "wikipedia_parallel_single_phase_index_test" + "dataSource": "wikipedia_parallel_index_test" }, "expectedResults":[ { @@ -20,7 +20,7 @@ "description":"having spec on post aggregation", "query":{ "queryType":"groupBy", - "dataSource":"wikipedia_parallel_single_phase_index_test", + "dataSource":"wikipedia_parallel_index_test", "granularity":"day", "dimensions":[ "page" diff --git a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json similarity index 92% rename from integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json rename to integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json index 48a8df2c15c3..911adbd9f277 100644 --- a/integration-tests/src/test/resources/indexer/wikipedia_parallel_single_phase_index_task.json +++ b/integration-tests/src/test/resources/indexer/wikipedia_parallel_index_task.json @@ -1,8 +1,8 @@ { - "type": "index_single_phase_parallel", + "type": "index_parallel", "spec": { "dataSchema": { - "dataSource": "wikipedia_parallel_single_phase_index_test", + "dataSource": "wikipedia_parallel_index_test", "metricsSpec": [ { "type": "count", @@ -55,7 +55,7 @@ } }, "ioConfig": { - "type": "index_single_phase_parallel", + "type": "index_parallel", "firehose": { "type": "local", "baseDir": "/resources/data/batch_index", diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 3e0f259bd3cf..69d373058a13 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -42,7 +42,7 @@ import io.druid.guice.annotations.Self; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.IndexTaskClientFactory; -import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClient; +import io.druid.indexing.common.task.ParallelIndexTaskClient; import io.druid.indexing.overlord.ForkingTaskRunner; import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.worker.Worker; @@ -96,7 +96,7 @@ public void configure(Binder binder) binder.bind(ForkingTaskRunner.class).in(LazySingleton.class); binder.bind(IndexingServiceClient.class).toProvider(Providers.of(null)); - binder.bind(new TypeLiteral>(){}) + binder.bind(new TypeLiteral>(){}) .toProvider(Providers.of(null)); binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(null)); diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index a2035847ccb1..a232bcba3304 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -56,7 +56,7 @@ import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.common.task.IndexTaskClientFactory; -import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClient; +import io.druid.indexing.common.task.ParallelIndexTaskClient; import io.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import io.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; import io.druid.indexing.overlord.ForkingTaskRunnerFactory; @@ -186,7 +186,7 @@ public void configure(Binder binder) binder.bind(SupervisorManager.class).in(LazySingleton.class); binder.bind(IndexingServiceClient.class).toProvider(Providers.of(null)); - binder.bind(new TypeLiteral>(){}) + binder.bind(new TypeLiteral>(){}) .toProvider(Providers.of(null)); binder.bind(ChatHandlerProvider.class).toProvider(Providers.of(null)); diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 072b8c5e42b2..0c79f8b5da74 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -66,8 +66,8 @@ import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.indexing.common.task.IndexTaskClientFactory; -import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClient; -import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskClientFactory; +import io.druid.indexing.common.task.ParallelIndexTaskClient; +import io.druid.indexing.common.task.ParallelIndexTaskClientFactory; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.HeapMemoryTaskStorage; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -190,8 +190,8 @@ public void configure(Binder binder) configureTaskActionClient(binder); binder.bind(IndexingServiceClient.class).to(HttpIndexingServiceClient.class).in(LazySingleton.class); - binder.bind(new TypeLiteral>(){}) - .to(SinglePhaseParallelIndexTaskClientFactory.class) + binder.bind(new TypeLiteral>(){}) + .to(ParallelIndexTaskClientFactory.class) .in(LazySingleton.class); binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); From 00e266347b0e3f4a2a6593b59263fc7df7fd5cf4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 6 Jun 2018 17:31:48 -0700 Subject: [PATCH 32/47] fix teamcity --- .../java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index 848d16ab1077..26d22b0f2f46 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -28,6 +28,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.druid.indexer.TaskLocation; +import io.druid.indexing.common.IndexTaskClient; import io.druid.indexing.common.TaskInfoProvider; import io.druid.indexing.common.TaskStatus; import io.druid.jackson.DefaultObjectMapper; @@ -157,7 +158,7 @@ public void testNoTaskLocation() @Test public void testTaskNotRunnableException() { - expectedException.expect(KafkaIndexTaskClient.TaskNotRunnableException.class); + expectedException.expect(IndexTaskClient.TaskNotRunnableException.class); expectedException.expectMessage("Aborting request because task [test-id] is not runnable"); reset(taskInfoProvider); From 86b45825231fae197e20e5e717a43c1d327e9785 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 28 Jun 2018 12:17:58 -0700 Subject: [PATCH 33/47] address comments --- .../data/input/FiniteFirehoseFactory.java | 10 ++++--- .../java/io/druid/data/input/InputSplit.java | 1 + docs/content/ingestion/native_tasks.md | 27 ++++++++++++------- .../firehose/s3/StaticS3FirehoseFactory.java | 7 +++++ 4 files changed, 33 insertions(+), 12 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java index 9e4bd7fba40e..6b768499ad9a 100644 --- a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java @@ -26,7 +26,7 @@ import java.util.stream.Stream; /** - * {@link FirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is + * {@link FiniteFirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is * limited. * * @param parser type @@ -35,7 +35,7 @@ public interface FiniteFirehoseFactory extends FirehoseFactory { /** - * Returns true if the {@link FirehoseFactory} supports parallel batch indexing. + * Returns true if this {@link FiniteFirehoseFactory} supports parallel batch indexing. */ @JsonIgnore @Override @@ -45,7 +45,11 @@ default boolean isSplittable() } /** - * Returns an iterator of {@link InputSplit}s. + * Returns a {@link Stream} for {@link InputSplit}s. In parallel batch indexing, each {@link InputSplit} is processed + * by a sub task. + * + * Listing splits may cause high overhead in some implementations. In this case, {@link InputSplit}s should be listed + * lazily so that the listing overhead could be amortized. */ @JsonIgnore Stream> getSplits() throws IOException; diff --git a/api/src/main/java/io/druid/data/input/InputSplit.java b/api/src/main/java/io/druid/data/input/InputSplit.java index 544964a60602..e8fd5099d5a7 100644 --- a/api/src/main/java/io/druid/data/input/InputSplit.java +++ b/api/src/main/java/io/druid/data/input/InputSplit.java @@ -24,6 +24,7 @@ /** * Input unit for distributed batch ingestion. Used in {@link FiniteFirehoseFactory}. + * An {@link InputSplit} represents the input data processed by a {@code io.druid.indexing.common.task.Task}. */ public class InputSplit { diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 2c15f358ef33..8789c50af4cd 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -134,7 +134,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no| |maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| -|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|150000|no| +|maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|150000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| @@ -460,14 +460,14 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no| |maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no| -|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|20000000|no| +|maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|20000000|no| |numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no| |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| |forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| -|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| +|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment pushing modes__ section.|false|no| |reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| -|publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no| +|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no| |segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no| #### IndexSpec @@ -497,12 +497,21 @@ For Roaring bitmaps: |type|String|Must be `roaring`.|yes| |compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)| -#### Segment publishing modes +#### Segment pushing modes -While ingesting data using the Index task, it creates segments from the input data and publishes them. For segment publishing, the Index task supports two segment publishing modes, i.e., _bulk publishing mode_ and _incremental publishing mode_ for [perfect rollup and best-effort rollup](./design/index.html), respectively. +While ingesting data using the Index task, it creates segments from the input data and pushes them. For segment pushing, +the Index task supports two segment pushing modes, i.e., _bulk pushing mode_ and _incremental pushing mode_ for +[perfect rollup and best-effort rollup](./design/index.html), respectively. -In the bulk publishing mode, every segment is published at the very end of the index task. Until then, created segments are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a problem due to limited storage capacity, and is not recommended to use in production. +In the bulk pushing mode, every segment is pushed at the very end of the index task. Until then, created segments +are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a +problem due to limited storage capacity, and is not recommended to use in production. -On the contrary, in the incremental publishing mode, segments are incrementally published, that is they can be published in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds, the index task immediately publishes all segments created until that moment, cleans all published segments up, and continues to ingest remaining data. +On the contrary, in the incremental pushing mode, segments are incrementally pushed, that is they can be pushed +in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory +and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds, +the index task immediately pushes all segments created until that moment, cleans all pushed segments up, and +continues to ingest remaining data. -To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig. +To enable bulk pushing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot +be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig. diff --git a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 11304521fb5e..a55272edb0ab 100644 --- a/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java +++ b/extensions-core/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -274,6 +274,13 @@ public FiniteFirehoseFactory withSplit(InputSplit + * {@code s3://{BUCKET_NAME}/{OBJECT_KEY}} + * + */ private static URI toUri(S3ObjectSummary object) { final String originalAuthority = object.getBucketName(); From 6cd0dd6bfc712e0c7688370452c88d8873b11ddb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Jul 2018 18:03:18 -0700 Subject: [PATCH 34/47] Fix license --- .../io/druid/data/input/FiniteFirehoseFactory.java | 12 ++++++------ .../main/java/io/druid/data/input/InputSplit.java | 12 ++++++------ .../druid/indexing/kafka/KafkaIndexTaskClient.java | 6 ++---- .../indexing/kafka/KafkaIndexTaskClientTest.java | 3 ++- .../CountingActionBasedSegmentAllocator.java | 12 ++++++------ .../SegmentAllocateActionGenerator.java | 13 +++++++------ .../java/io/druid/indexing/common/Counters.java | 12 ++++++------ .../io/druid/indexing/common/IndexTaskClient.java | 12 ++++++------ .../actions/CountingSegmentAllocateAction.java | 12 ++++++------ .../indexing/common/actions/SurrogateAction.java | 13 +++++++------ .../common/task/ClientBasedTaskInfoProvider.java | 12 ++++++------ .../common/task/IndexTaskClientFactory.java | 12 ++++++------ .../indexing/common/task/ParallelIndexIOConfig.java | 12 ++++++------ .../common/task/ParallelIndexIngestionSpec.java | 12 ++++++------ .../indexing/common/task/ParallelIndexSubTask.java | 12 ++++++------ .../common/task/ParallelIndexSubTaskSpec.java | 12 ++++++------ .../common/task/ParallelIndexSupervisorTask.java | 12 ++++++------ .../common/task/ParallelIndexTaskClient.java | 12 ++++++------ .../common/task/ParallelIndexTaskClientFactory.java | 12 ++++++------ .../common/task/ParallelIndexTaskRunner.java | 13 +++++++------ .../common/task/ParallelIndexTuningConfig.java | 12 ++++++------ .../indexing/common/task/PushedSegmentsReport.java | 12 ++++++------ .../task/SinglePhaseParallelIndexTaskRunner.java | 13 +++++++------ .../io/druid/indexing/common/task/SubTaskSpec.java | 12 ++++++------ .../io/druid/indexing/common/task/TaskMonitor.java | 12 ++++++------ .../actions/CountingSegmentAllocateActionTest.java | 12 ++++++------ .../common/actions/SurrogateActionTest.java | 12 ++++++------ .../AbstractParallelIndexSupervisorTaskTest.java | 13 +++++++------ .../indexing/common/task/IngestionTestBase.java | 12 ++++++------ .../ParallelIndexSupervisorTaskResourceTest.java | 13 +++++++------ .../task/ParallelIndexSupervisorTaskSerdeTest.java | 12 ++++++------ .../task/ParallelIndexSupervisorTaskTest.java | 12 ++++++------ .../druid/indexing/common/task/TaskMonitorTest.java | 12 ++++++------ .../tests/indexer/AbstractITBatchIndexTest.java | 13 +++++++------ .../io/druid/tests/indexer/ITParallelIndexTest.java | 13 +++++++------ .../java/io/druid/client/indexing/ClientQuery.java | 12 ++++++------ .../client/indexing/HttpIndexingServiceClient.java | 12 ++++++------ .../java/io/druid/client/indexing/TaskStatus.java | 13 +++++++------ .../druid/client/indexing/TaskStatusResponse.java | 12 ++++++------ .../segment/realtime/firehose/ChatHandlers.java | 12 ++++++------ .../client/indexing/NoopIndexingServiceClient.java | 12 ++++++------ 41 files changed, 247 insertions(+), 239 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java index 6b768499ad9a..bb62161e6710 100644 --- a/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java +++ b/api/src/main/java/io/druid/data/input/FiniteFirehoseFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/api/src/main/java/io/druid/data/input/InputSplit.java b/api/src/main/java/io/druid/data/input/InputSplit.java index e8fd5099d5a7..b3886b630911 100644 --- a/api/src/main/java/io/druid/data/input/InputSplit.java +++ b/api/src/main/java/io/druid/data/input/InputSplit.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index 7c2259e285cc..046e2992302a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -19,6 +19,7 @@ package io.druid.indexing.kafka; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; @@ -283,7 +284,7 @@ public boolean setEndOffsets( final String id, final Map endOffsets, final boolean finalize - ) + ) throws IOException { log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize); @@ -301,9 +302,6 @@ public boolean setEndOffsets( catch (NoTaskLocationException e) { return false; } - catch (IOException e) { - throw Throwables.propagate(e); - } } public ListenableFuture stopAsync(final String id, final boolean publish) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java index 8804066f5a89..bfa0d89aa987 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskClientTest.java @@ -57,6 +57,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.io.IOException; import java.net.URL; import java.util.Collections; import java.util.List; @@ -135,7 +136,7 @@ public void tearDown() } @Test - public void testNoTaskLocation() + public void testNoTaskLocation() throws IOException { reset(taskInfoProvider); expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes(); diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java index 77c2b901b075..e0a92167edba 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java index 768815fa6e06..892524897dbc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.indexing.appenderator; import io.druid.data.input.InputRow; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/Counters.java b/indexing-service/src/main/java/io/druid/indexing/common/Counters.java index 8b8b2ac8abb6..6563c7db9af5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/Counters.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/Counters.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java index ed2db4abe6a5..a3aa4ab498e0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java index 2d1a7b8a497b..c4977d3ea67c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java index 7acbddd7cc48..8befeb0e5cea 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/SurrogateAction.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java index 54daa3016eac..243a7ad06d40 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ClientBasedTaskInfoProvider.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java index 85bf33058e19..54b0c3272268 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTaskClientFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java index 1ef2d62b04b8..18d4709c9c96 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java index bf76ef6dc7f1..34fae34729c6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java index 255d0acb11e4..635700ee964b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java index 156c776a897d..0210e6f5340c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java index 227506625a45..1b1fbee9a299 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java index 23a4ce71385d..9d2f0dda4d90 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java index 8ce2a3fd2370..494160db40ec 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java index 90d02c2122f6..c4672c7c93a6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.indexing.common.task; import io.druid.indexer.TaskState; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java index 0d027e39f39a..f060c4751c4a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java b/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java index f5a209381a4c..950a6547b1c8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java index 0dc3119b9c17..19cd44680246 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JacksonInject; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java index 0d9cb76fee10..9d9c62b658c9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index 639580fd7340..b7fabde5f0ae 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java index 2cb8d8a1ee08..ab702b737add 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java index 9311c92aba89..b21a82b76e5f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/actions/SurrogateActionTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java index d229e6562c3d..9ae4dff0fedd 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.indexing.common.task; import com.google.common.base.Throwables; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java index d8e5531aca7a..c9071fca5b76 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IngestionTestBase.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java index aeb2cefaaac4..cc43eb8912e4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.indexing.common.task; import com.google.common.base.Function; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java index 766ad93116ad..3a8f07ef843d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java index 5e837216f882..b8b9eab31a64 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java index 44e222c6e9f5..7020b4509b9a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java index 661018f5529b..74cfb7e3ac34 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.tests.indexer; import com.google.inject.Inject; diff --git a/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java index a37716d617b2..0dff81e4cc91 100644 --- a/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java +++ b/integration-tests/src/test/java/io/druid/tests/indexer/ITParallelIndexTest.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.tests.indexer; import io.druid.testing.guice.DruidTestModuleFactory; diff --git a/server/src/main/java/io/druid/client/indexing/ClientQuery.java b/server/src/main/java/io/druid/client/indexing/ClientQuery.java index f692e3c2d597..a412bc8b349d 100644 --- a/server/src/main/java/io/druid/client/indexing/ClientQuery.java +++ b/server/src/main/java/io/druid/client/indexing/ClientQuery.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java index ab52eb16c3c5..b012db604426 100644 --- a/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/HttpIndexingServiceClient.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/server/src/main/java/io/druid/client/indexing/TaskStatus.java b/server/src/main/java/io/druid/client/indexing/TaskStatus.java index 4a17629709cc..708106d2bb99 100644 --- a/server/src/main/java/io/druid/client/indexing/TaskStatus.java +++ b/server/src/main/java/io/druid/client/indexing/TaskStatus.java @@ -1,21 +1,22 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ + package io.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java index 7f7ba6e9b057..f9c5191a0a07 100644 --- a/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java +++ b/server/src/main/java/io/druid/client/indexing/TaskStatusResponse.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java b/server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java index 350bbf667c9f..c3ba05bcb367 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/ChatHandlers.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java b/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java index 0c80d2b6a8d1..030c991cbfef 100644 --- a/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java +++ b/server/src/test/java/io/druid/client/indexing/NoopIndexingServiceClient.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * 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. Metamarkets licenses this file + * 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ From 509c9b3cb406133cc25c2847ed9d7d6745df9454 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Jul 2018 18:31:23 -0700 Subject: [PATCH 35/47] addressing comments --- .../io/druid/indexing/kafka/KafkaIndexTaskModule.java | 8 +++++--- .../appenderator/SegmentAllocateActionGenerator.java | 8 ++++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java index 443b8a904c83..3771d9291483 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskModule.java @@ -53,8 +53,10 @@ public List getJacksonModules() @Override public void configure(Binder binder) { - binder.bind(new TypeLiteral>(){}) - .to(KafkaIndexTaskClientFactory.class) - .in(LazySingleton.class); + binder.bind( + new TypeLiteral>() + { + } + ).to(KafkaIndexTaskClientFactory.class).in(LazySingleton.class); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java index 892524897dbc..4a0352f9b8d0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java +++ b/indexing-service/src/main/java/io/druid/indexing/appenderator/SegmentAllocateActionGenerator.java @@ -24,6 +24,14 @@ import io.druid.segment.indexing.DataSchema; import io.druid.segment.realtime.appenderator.SegmentIdentifier; +/** + * This class is used in {@link ActionBasedSegmentAllocator} and expected to generate a + * {@link TaskAction} which is submitted to overlords to allocate a new segment. + * The {@link #generate} method can return any implementation of {@link TaskAction}. + * + * @see io.druid.indexing.common.actions.SegmentAllocateAction + * @see io.druid.indexing.common.actions.SurrogateAction + */ public interface SegmentAllocateActionGenerator { TaskAction generate( From 6a1bcfdb5ff4499a3f84d347e05c55d11fbbdca8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 11 Jul 2018 19:29:45 -0700 Subject: [PATCH 36/47] addressing comments --- .../indexing/kafka/KafkaIndexTaskClient.java | 2 -- .../common/task/ParallelIndexSubTask.java | 17 ++++++----------- .../task/ParallelIndexSupervisorTask.java | 2 +- .../common/task/ParallelIndexTaskClient.java | 4 ++-- .../common/task/PushedSegmentsReport.java | 5 +++++ 5 files changed, 14 insertions(+), 16 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java index 046e2992302a..7e65d7fea040 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClient.java @@ -19,10 +19,8 @@ package io.druid.indexing.kafka; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListenableFuture; import io.druid.indexing.common.IndexTaskClient; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java index 635700ee964b..e839a3cce6e2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java @@ -24,8 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import io.druid.client.indexing.IndexingServiceClient; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -64,14 +62,14 @@ import io.druid.segment.realtime.appenderator.SegmentAllocator; import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; import io.druid.timeline.DataSegment; -import org.codehaus.plexus.util.FileUtils; +import org.apache.commons.io.FileUtils; import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.SortedSet; @@ -230,23 +228,20 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } - private Map tryAcquireExclusiveSurrogateLocks( + private void tryAcquireExclusiveSurrogateLocks( TaskActionClient client, SortedSet intervals ) throws IOException { - final Map lockMap = new HashMap<>(); for (Interval interval : Tasks.computeCompactIntervals(intervals)) { - final TaskLock lock = Preconditions.checkNotNull( + Preconditions.checkNotNull( client.submit( new SurrogateAction<>(supervisorTaskId, new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) ), "Cannot acquire a lock for interval[%s]", interval ); - lockMap.put(interval, lock); } - return lockMap; } /** @@ -284,8 +279,8 @@ dataSchema, new RealtimeIOConfig(null, null, null), null if (toolbox.getMonitorScheduler() != null) { toolbox.getMonitorScheduler().addMonitor( new RealtimeMetricsMonitor( - ImmutableList.of(fireDepartmentForMetrics), - ImmutableMap.of(DruidMetrics.TASK_ID, new String[]{getId()}) + Collections.singletonList(fireDepartmentForMetrics), + Collections.singletonMap(DruidMetrics.TASK_ID, new String[]{getId()}) ) ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java index 1b1fbee9a299..b9de88ea546f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java @@ -176,7 +176,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception static boolean isReady(TaskActionClient actionClient, SortedSet intervals) throws IOException { final List locks = getTaskLocks(actionClient); - if (locks.size() == 0) { + if (locks.isEmpty()) { try { Tasks.tryAcquireExclusiveLocks(actionClient, intervals); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java index 9d2f0dda4d90..5d06a5fa315b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java @@ -36,7 +36,7 @@ public class ParallelIndexTaskClient extends IndexTaskClient { private final String subtaskId; - public ParallelIndexTaskClient( + ParallelIndexTaskClient( HttpClient httpClient, ObjectMapper objectMapper, TaskInfoProvider taskInfoProvider, @@ -49,7 +49,7 @@ public ParallelIndexTaskClient( this.subtaskId = callerId; } - public String getSubtaskId() + String getSubtaskId() { return subtaskId; } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java b/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java index 950a6547b1c8..df2cb7eba3ff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java @@ -26,6 +26,11 @@ import java.util.List; +/** + * This class is used in native parallel batch indexing, currently only in {@link SinglePhaseParallelIndexTaskRunner}. + * In native parallel batch indexing, each subTask generates and pushes segments and sends a report to the + * supervisorTask. Once the supervisorTask collects all reports, it publishes all the pushed segments at once. + */ public class PushedSegmentsReport { private final String taskId; From d700395f9a2760a6b215797319a3e7e8ef81db6b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 13 Jul 2018 16:57:03 -0700 Subject: [PATCH 37/47] indexTaskClient-based segmentAllocator instead of CountingActionBasedSegmentAllocator --- .../CountingActionBasedSegmentAllocator.java | 74 ------- .../CountingSegmentAllocateAction.java | 196 ------------------ .../indexing/common/actions/TaskAction.java | 1 - .../common/task/ParallelIndexSubTask.java | 101 ++++----- .../task/ParallelIndexSupervisorTask.java | 7 +- .../common/task/ParallelIndexTaskClient.java | 30 +++ .../common/task/ParallelIndexTaskRunner.java | 3 +- .../SinglePhaseParallelIndexTaskRunner.java | 121 ++++++++++- .../CountingSegmentAllocateActionTest.java | 143 ------------- ...stractParallelIndexSupervisorTaskTest.java | 25 ++- ...rallelIndexSupervisorTaskResourceTest.java | 9 +- .../task/ParallelIndexSupervisorTaskTest.java | 7 +- 12 files changed, 224 insertions(+), 493 deletions(-) delete mode 100644 indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java delete mode 100644 indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java delete mode 100644 indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java b/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java deleted file mode 100644 index e0a92167edba..000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/appenderator/CountingActionBasedSegmentAllocator.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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 io.druid.indexing.appenderator; - -import com.google.common.base.Preconditions; -import io.druid.data.input.InputRow; -import io.druid.indexing.common.actions.CountingSegmentAllocateAction; -import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.segment.indexing.granularity.GranularitySpec; -import io.druid.segment.realtime.appenderator.SegmentAllocator; -import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import org.joda.time.Interval; - -import java.io.IOException; -import java.util.Map; - -/** - * Segment allocator based on {@link CountingSegmentAllocateAction}. - */ -public class CountingActionBasedSegmentAllocator implements SegmentAllocator -{ - private final TaskActionClient taskActionClient; - private final String dataSource; - private final GranularitySpec granularitySpec; - private final Map versions; - - public CountingActionBasedSegmentAllocator( - TaskActionClient taskActionClient, - String dataSource, - GranularitySpec granularitySpec, - Map versions - ) - { - this.taskActionClient = Preconditions.checkNotNull(taskActionClient, "taskActionClient"); - this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); - this.granularitySpec = Preconditions.checkNotNull(granularitySpec, "granularitySpec"); - this.versions = Preconditions.checkNotNull(versions, "versions"); - } - - @Override - public SegmentIdentifier allocate( - InputRow row, - String sequenceName, - String previousSegmentId, - boolean skipSegmentLineageCheck - ) throws IOException - { - return taskActionClient.submit( - new CountingSegmentAllocateAction( - dataSource, - row.getTimestamp(), - granularitySpec, - versions - ) - ); - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java deleted file mode 100644 index c4977d3ea67c..000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/CountingSegmentAllocateAction.java +++ /dev/null @@ -1,196 +0,0 @@ -/* - * 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 io.druid.indexing.common.actions; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.KeyDeserializer; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import io.druid.indexing.common.Counters; -import io.druid.indexing.common.task.Task; -import io.druid.java.util.common.ISE; -import io.druid.java.util.common.Intervals; -import io.druid.segment.indexing.granularity.GranularitySpec; -import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import io.druid.timeline.partition.NumberedShardSpec; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; -import java.util.SortedSet; - -/** - * This action is to find a proper {@link io.druid.timeline.partition.ShardSpec} based on counting. This keeps track of - * the next shard number per {@link Interval} in {@link Counters}. The next shard number is incremented by 1 whenever a - * new {@link SegmentIdentifier} is allocated. - */ -public class CountingSegmentAllocateAction implements TaskAction -{ - private final String dataSource; - private final DateTime timestamp; - private final GranularitySpec granularitySpec; - @JsonDeserialize(keyUsing = IntervalDeserializer.class) - private final Map versions; - - private final SortedSet bucketIntervals; - - @JsonCreator - public CountingSegmentAllocateAction( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("timestamp") DateTime timestamp, - @JsonProperty("granularitySpec") GranularitySpec granularitySpec, - @JsonProperty("versions") Map versions - ) - { - this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); - this.timestamp = Preconditions.checkNotNull(timestamp, "timestamp"); - this.granularitySpec = Preconditions.checkNotNull(granularitySpec, "granularitySpec"); - this.versions = Preconditions.checkNotNull(versions, "versions"); - - this.bucketIntervals = Preconditions.checkNotNull(granularitySpec.bucketIntervals().orNull(), "bucketIntervals"); - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public DateTime getTimestamp() - { - return timestamp; - } - - @JsonProperty - public GranularitySpec getGranularitySpec() - { - return granularitySpec; - } - - @JsonProperty - public Map getVersions() - { - return versions; - } - - @Override - public TypeReference getReturnTypeReference() - { - return new TypeReference() - { - }; - } - - @Override - public SegmentIdentifier perform(Task task, TaskActionToolbox toolbox) - { - Optional maybeInterval = granularitySpec.bucketInterval(timestamp); - if (!maybeInterval.isPresent()) { - throw new ISE("Could not find interval for timestamp [%s]", timestamp); - } - - final Interval interval = maybeInterval.get(); - if (!bucketIntervals.contains(interval)) { - throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); - } - - final Counters counters = toolbox.getCounters(); - - final int partitionNum = counters.increment(interval.toString(), 1); - return new SegmentIdentifier( - dataSource, - interval, - findVersion(versions, interval), - new NumberedShardSpec(partitionNum, 0) - ); - } - - @Override - public boolean isAudited() - { - return false; - } - - @Override - public String toString() - { - return "CountingSegmentAllocateAction{" + - "dataSource='" + dataSource + '\'' + - ", timestamp=" + timestamp + - ", granularitySpec=" + granularitySpec + - ", versions=" + versions + - '}'; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - final CountingSegmentAllocateAction that = (CountingSegmentAllocateAction) o; - if (!dataSource.equals(that.dataSource)) { - return false; - } - if (!timestamp.equals(that.timestamp)) { - return false; - } - if (!granularitySpec.equals(that.granularitySpec)) { - return false; - } - return versions.equals(that.versions); - } - - @Override - public int hashCode() - { - return Objects.hash(dataSource, timestamp, granularitySpec, versions); - } - - private static String findVersion(Map versions, Interval interval) - { - return versions.entrySet().stream() - .filter(entry -> entry.getKey().contains(interval)) - .map(Entry::getValue) - .findFirst() - .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); - } - - public static class IntervalDeserializer extends KeyDeserializer - { - @Override - public Object deserializeKey(String s, DeserializationContext deserializationContext) - { - return Intervals.of(s); - } - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index 2b1be99ebfa0..fe1ae7967930 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -37,7 +37,6 @@ @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), @JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class), @JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class), - @JsonSubTypes.Type(name = "countingSegmentAllocate", value = CountingSegmentAllocateAction.class), @JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class), @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class), @JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java index e839a3cce6e2..1fca8b4df9e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java @@ -31,16 +31,12 @@ import io.druid.indexer.TaskStatus; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; -import io.druid.indexing.appenderator.CountingActionBasedSegmentAllocator; -import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskLockType; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.common.task.IndexTask.IndexIOConfig; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; @@ -75,7 +71,6 @@ import java.util.SortedSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; /** * A worker task of {@link ParallelIndexSupervisorTask}. Similar to {@link IndexTask}, but this task @@ -182,11 +177,6 @@ public String getSupervisorTaskId() @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { - final boolean explicitIntervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals() - .isPresent(); - final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { @@ -198,24 +188,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception // Firehose temporary directory is automatically removed when this IndexTask completes. FileUtils.forceMkdir(firehoseTempDir); - final DataSchema dataSchema = ingestionSchema.getDataSchema(); - final Map versions; - - if (explicitIntervals) { - versions = toolbox.getTaskActionClient().submit(new SurrogateAction<>(supervisorTaskId, new LockListAction())) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - } else { - versions = null; - } - - final List pushedSegments = generateAndPushSegments( - toolbox, - dataSchema, - versions, - firehoseFactory, - firehoseTempDir - ); final ParallelIndexTaskClient taskClient = taskClientFactory.build( new ClientBasedTaskInfoProvider(indexingServiceClient), getId(), @@ -223,6 +195,12 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); + final List pushedSegments = generateAndPushSegments( + toolbox, + taskClient, + firehoseFactory, + firehoseTempDir + ); taskClient.report(supervisorTaskId, pushedSegments); return TaskStatus.success(getId()); @@ -244,6 +222,40 @@ private void tryAcquireExclusiveSurrogateLocks( } } + private SegmentAllocator createSegmentAllocator( + TaskToolbox toolbox, + ParallelIndexTaskClient taskClient, + ParallelIndexIngestionSpec ingestionSchema + ) + { + final DataSchema dataSchema = ingestionSchema.getDataSchema(); + final boolean explicitIntervals = dataSchema.getGranularitySpec().bucketIntervals().isPresent(); + final ParallelIndexIOConfig ioConfig = ingestionSchema.getIOConfig(); + if (ioConfig.isAppendToExisting() || !explicitIntervals) { + return new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SurrogateAction<>( + supervisorTaskId, + new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck + ) + ) + ); + } else { + return (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> taskClient.allocateSegment( + supervisorTaskId, + row.getTimestamp() + ); + } + } + /** * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}. * If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs @@ -264,12 +276,12 @@ private void tryAcquireExclusiveSurrogateLocks( */ private List generateAndPushSegments( final TaskToolbox toolbox, - final DataSchema dataSchema, - @Nullable final Map versions, + final ParallelIndexTaskClient taskClient, final FirehoseFactory firehoseFactory, final File firehoseTempDir ) throws IOException, InterruptedException { + final DataSchema dataSchema = ingestionSchema.getDataSchema(); final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); final FireDepartment fireDepartmentForMetrics = new FireDepartment( dataSchema, new RealtimeIOConfig(null, null, null), null @@ -285,37 +297,10 @@ dataSchema, new RealtimeIOConfig(null, null, null), null ); } - final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final long pushTimeout = tuningConfig.getPushTimeout(); final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); - - final SegmentAllocator segmentAllocator; - if (ioConfig.isAppendToExisting() || !explicitIntervals) { - segmentAllocator = new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SurrogateAction<>( - supervisorTaskId, - new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ) - ); - } else { - segmentAllocator = new CountingActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - getDataSource(), - granularitySpec, - versions - ); - } + final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient, ingestionSchema); try ( final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java index b9de88ea546f..372fccdd8196 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java @@ -145,12 +145,13 @@ AuthorizerMapper getAuthorizerMapper() } @VisibleForTesting - ParallelIndexTaskRunner createRunner() + ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) { if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { throw new UnsupportedOperationException("Perfect roll-up is not supported yet"); } else { runner = new SinglePhaseParallelIndexTaskRunner( + toolbox, getId(), getGroupId(), ingestionSchema, @@ -204,8 +205,8 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception private TaskStatus runParallel(TaskToolbox toolbox) throws Exception { - createRunner(); - return TaskStatus.fromCode(getId(), runner.run(toolbox)); + createRunner(toolbox); + return TaskStatus.fromCode(getId(), runner.run()); } private TaskStatus runSequential(TaskToolbox toolbox) throws Exception diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java index 5d06a5fa315b..97785b166ecd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java @@ -19,14 +19,17 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.indexing.common.IndexTaskClient; import io.druid.indexing.common.TaskInfoProvider; import io.druid.java.util.common.ISE; import io.druid.java.util.http.client.HttpClient; import io.druid.java.util.http.client.response.FullResponseHolder; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.DateTime; import org.joda.time.Duration; import java.io.IOException; @@ -54,6 +57,33 @@ String getSubtaskId() return subtaskId; } + public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException + { + final FullResponseHolder response = submitSmileRequest( + supervisorTaskId, + HttpMethod.POST, + "segment/allocate", + null, + serialize(timestamp), + true + ); + if (!isSuccess(response)) { + throw new ISE( + "task[%s] failed to allocate a new segment identifier with the HTTP code[%d] and content[%s]", + supervisorTaskId, + response.getStatus().getCode(), + response.getContent() + ); + } else { + return deserialize( + response.getContent(), + new TypeReference() + { + } + ); + } + } + public void report(String supervisorTaskId, List pushedSegments) { try { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java index c4672c7c93a6..92674c5fbea4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java @@ -20,7 +20,6 @@ package io.druid.indexing.common.task; import io.druid.indexer.TaskState; -import io.druid.indexing.common.TaskToolbox; /** * ParallelIndexTaskRunner is the actual task runner of {@link ParallelIndexSupervisorTask}. There is currently a single @@ -29,5 +28,5 @@ */ public interface ParallelIndexTaskRunner { - TaskState run(TaskToolbox toolbox) throws Exception; + TaskState run() throws Exception; } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java index 19cd44680246..db6a00171b7f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java @@ -19,12 +19,13 @@ package io.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -35,14 +36,19 @@ import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import io.druid.indexing.common.Counters; +import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status; import io.druid.indexing.common.task.TaskMonitor.MonitorEntry; import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; import io.druid.indexing.common.task.TaskMonitor.TaskHistory; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import io.druid.segment.realtime.appenderator.UsedSegmentChecker; @@ -52,6 +58,9 @@ import io.druid.server.security.Action; import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; @@ -71,7 +80,9 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; +import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -91,6 +102,7 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn { private static final Logger log = new Logger(SinglePhaseParallelIndexTaskRunner.class); + private final TaskToolbox toolbox; private final String taskId; private final String groupId; private final ParallelIndexIngestionSpec ingestionSchema; @@ -107,21 +119,25 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn // subTaskId -> report private final ConcurrentMap segmentsMap = new ConcurrentHashMap<>(); + private final Counters counters = new Counters(); + private volatile boolean stopped; private volatile TaskMonitor taskMonitor; private int nextSpecId = 0; SinglePhaseParallelIndexTaskRunner( - @JsonProperty("id") String taskId, + TaskToolbox toolbox, + String taskId, String groupId, - @JsonProperty("spec") ParallelIndexIngestionSpec ingestionSchema, - @JsonProperty("context") Map context, - @JacksonInject IndexingServiceClient indexingServiceClient, - @JacksonInject ChatHandlerProvider chatHandlerProvider, - @JacksonInject AuthorizerMapper authorizerMapper + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient, + ChatHandlerProvider chatHandlerProvider, + AuthorizerMapper authorizerMapper ) { + this.toolbox = toolbox; this.taskId = taskId; this.groupId = groupId; this.ingestionSchema = ingestionSchema; @@ -134,7 +150,7 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn } @Override - public TaskState run(TaskToolbox toolbox) throws Exception + public TaskState run() throws Exception { log.info( "Found chat handler of class[%s]", @@ -255,6 +271,12 @@ public TaskState run(TaskToolbox toolbox) throws Exception return state; } + @VisibleForTesting + TaskToolbox getToolbox() + { + return toolbox; + } + @VisibleForTesting ParallelIndexIngestionSpec getIngestionSchema() { @@ -266,8 +288,87 @@ private String getDataSource() return ingestionSchema.getDataSchema().getDataSource(); } - // Internal API for collecting reports from subTasks + // Internal APIs + + /** + * Allocate a new {@link SegmentIdentifier} for a request from {@link ParallelIndexSubTask}. + * The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec}) + * per bucket interval. + */ + @POST + @Path("/segment/allocate") + @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) + public Response allocateSegment( + DateTime timestamp, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck( + req, + Action.READ, + getDataSource(), + authorizerMapper + ); + + try { + final SegmentIdentifier segmentIdentifier = allocateNewSegment(timestamp); + return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build(); + } + catch (IOException | IllegalStateException e) { + return Response.serverError().entity(Throwables.getStackTraceAsString(e)).build(); + } + catch (IllegalArgumentException e) { + return Response.status(Response.Status.BAD_REQUEST).entity(Throwables.getStackTraceAsString(e)).build(); + } + } + + @VisibleForTesting + SegmentIdentifier allocateNewSegment(DateTime timestamp) throws IOException + { + final String dataSource = getDataSource(); + final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec(); + final SortedSet bucketIntervals = Preconditions.checkNotNull( + granularitySpec.bucketIntervals().orNull(), + "bucketIntervals" + ); + // List locks whenever allocating a new segment because locks might be revoked and no longer valid. + final Map versions = toolbox + .getTaskActionClient() + .submit(new LockListAction()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + + final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); + if (!maybeInterval.isPresent()) { + throw new IAE("Could not find interval for timestamp [%s]", timestamp); + } + + final Interval interval = maybeInterval.get(); + if (!bucketIntervals.contains(interval)) { + throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); + } + + final int partitionNum = counters.increment(interval.toString(), 1); + return new SegmentIdentifier( + dataSource, + interval, + findVersion(versions, interval), + new NumberedShardSpec(partitionNum, 0) + ); + } + + private static String findVersion(Map versions, Interval interval) + { + return versions.entrySet().stream() + .filter(entry -> entry.getKey().contains(interval)) + .map(Entry::getValue) + .findFirst() + .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); + } + /** + * {@link ParallelIndexSubTask}s call this API to report the segments they've generated and pushed. + */ @POST @Path("/report") @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE) @@ -283,7 +384,7 @@ public Response report( authorizerMapper ); collectReport(report); - return Response.status(Response.Status.OK).build(); + return Response.ok().build(); } @VisibleForTesting diff --git a/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java b/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java deleted file mode 100644 index ab702b737add..000000000000 --- a/indexing-service/src/test/java/io/druid/indexing/common/actions/CountingSegmentAllocateActionTest.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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 io.druid.indexing.common.actions; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.druid.indexing.common.task.IngestionTestBase; -import io.druid.indexing.common.task.NoopTask; -import io.druid.indexing.common.task.Task; -import io.druid.jackson.DefaultObjectMapper; -import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.concurrent.Execs; -import io.druid.java.util.common.granularity.Granularities; -import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import io.druid.timeline.partition.NumberedShardSpec; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.Comparator; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -public class CountingSegmentAllocateActionTest extends IngestionTestBase -{ - @Test - public void testJsonSerde() throws IOException - { - final SimpleModule module = new SimpleModule(); - module.addSerializer(Interval.class, ToStringSerializer.instance); - - final ObjectMapper mapper = new DefaultObjectMapper(); - mapper.registerModule(module); - mapper.registerSubtypes(new NamedType(NumberedShardSpec.class, "numbered")); - - final CountingSegmentAllocateAction action = createAction(DateTimes.nowUtc()); - - final String json = mapper.writeValueAsString(action); - Assert.assertEquals(action, mapper.readValue(json, TaskAction.class)); - } - - @Test - public void testPerform() - { - final Task task = NoopTask.create(); - final TaskActionToolbox toolbox = createTaskActionToolbox(); - - CountingSegmentAllocateAction action; - SegmentIdentifier segmentIdentifier; - - for (int i = 0; i < 3; i++) { - action = createAction(DateTimes.of("2017-01-01")); - segmentIdentifier = action.perform(task, toolbox); - Assert.assertEquals( - StringUtils.format("dataSource_2017-01-01T00:00:00.000Z_2017-01-02T00:00:00.000Z_version1_%d", i + 1), - segmentIdentifier.toString() - ); - } - - for (int i = 0; i < 3; i++) { - action = createAction(DateTimes.of("2017-01-02")); - segmentIdentifier = action.perform(task, toolbox); - Assert.assertEquals( - StringUtils.format("dataSource_2017-01-02T00:00:00.000Z_2017-01-03T00:00:00.000Z_version1_%d", i + 1), - segmentIdentifier.toString() - ); - } - } - - @Test - public void testPerformParallel() - { - final List tasks = IntStream.range(0, 5).mapToObj(i -> NoopTask.create()).collect(Collectors.toList()); - final ExecutorService service = Execs.multiThreaded(5, "counting-segment-allocate-action-test-%d"); - final TaskActionToolbox toolbox = createTaskActionToolbox(); - - try { - final List segmentIdentifiers = tasks - .stream() - .map(task -> { - final CountingSegmentAllocateAction action = createAction(DateTimes.of("2017-01-01")); - return action.perform(task, toolbox); - }) - .sorted(Comparator.comparing(SegmentIdentifier::toString)) - .collect(Collectors.toList()); - for (int i = 0; i < 5; i++) { - Assert.assertEquals( - StringUtils.format("dataSource_2017-01-01T00:00:00.000Z_2017-01-02T00:00:00.000Z_version1_%d", i + 1), - segmentIdentifiers.get(i).toString() - ); - } - } - finally { - service.shutdownNow(); - } - } - - private CountingSegmentAllocateAction createAction(DateTime timestamp) - { - return new CountingSegmentAllocateAction( - "dataSource", - timestamp, - new ArbitraryGranularitySpec( - Granularities.DAY, - ImmutableList.of(Intervals.of("2017-01-01/2017-01-02"), Intervals.of("2017-01-02/2017-01-03")) - ), - ImmutableMap.of( - Intervals.of("2017-01-01/2017-01-02"), - "version1", - Intervals.of("2017-01-02/2017-01-03"), - "version1" - ) - ); - } -} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java index 9ae4dff0fedd..029289d74fce 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java @@ -48,12 +48,14 @@ import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusherConfig; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Rule; import org.junit.rules.TemporaryFolder; @@ -296,6 +298,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.fromCode( getId(), new TestParallelIndexTaskRunner( + toolbox, getId(), getGroupId(), getIngestionSchema(), @@ -303,7 +306,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception new NoopIndexingServiceClient(), new NoopChatHandlerProvider(), new AuthorizerMapper(Collections.emptyMap()) - ).run(toolbox) + ).run() ); } } @@ -311,6 +314,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRunner { TestParallelIndexTaskRunner( + TaskToolbox toolbox, String taskId, String groupId, ParallelIndexIngestionSpec ingestionSchema, @@ -320,7 +324,16 @@ static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRun AuthorizerMapper authorizerMapper ) { - super(taskId, groupId, ingestionSchema, context, indexingServiceClient, chatHandlerProvider, authorizerMapper); + super( + toolbox, + taskId, + groupId, + ingestionSchema, + context, + indexingServiceClient, + chatHandlerProvider, + authorizerMapper + ); } @Override @@ -368,12 +381,18 @@ static class LocalParallelIndexTaskClient extends ParallelIndexTaskClient { private final SinglePhaseParallelIndexTaskRunner runner; - public LocalParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexTaskRunner runner) + LocalParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexTaskRunner runner) { super(null, null, null, null, callerId, 0); this.runner = runner; } + @Override + public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException + { + return runner.allocateNewSegment(timestamp); + } + @Override public void report(String supervisorTaskId, List pushedSegments) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java index cc43eb8912e4..4df477807dae 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java @@ -506,6 +506,7 @@ private class TestSupervisorTask extends TestParallelIndexSupervisorTask public TaskStatus run(TaskToolbox toolbox) throws Exception { this.runner = new TestRunner( + toolbox, this, indexingServiceClient, new NoopChatHandlerProvider(), @@ -513,7 +514,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception ); return TaskStatus.fromCode( getId(), - runner.run(toolbox) + runner.run() ); } @@ -529,6 +530,7 @@ private class TestRunner extends TestParallelIndexTaskRunner private final ParallelIndexSupervisorTask supervisorTask; TestRunner( + TaskToolbox toolbox, ParallelIndexSupervisorTask supervisorTask, @Nullable IndexingServiceClient indexingServiceClient, @Nullable ChatHandlerProvider chatHandlerProvider, @@ -536,6 +538,7 @@ private class TestRunner extends TestParallelIndexTaskRunner ) { super( + toolbox, supervisorTask.getId(), supervisorTask.getGroupId(), supervisorTask.getIngestionSchema(), @@ -554,6 +557,7 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) .getIOConfig() .getFirehoseFactory(); final TestSubTaskSpec spec = new TestSubTaskSpec( + toolbox, supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), supervisorTask, @@ -576,9 +580,11 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) private class TestSubTaskSpec extends ParallelIndexSubTaskSpec { + private final TaskToolbox toolbox; private final SinglePhaseParallelIndexTaskRunner runner; TestSubTaskSpec( + TaskToolbox toolbox, String id, String groupId, ParallelIndexSupervisorTask supervisorTask, @@ -589,6 +595,7 @@ private class TestSubTaskSpec extends ParallelIndexSubTaskSpec ) { super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit); + this.toolbox = toolbox; this.runner = runner; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java index b8b9eab31a64..cd845bef1131 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java @@ -106,7 +106,7 @@ public void testIsReady() throws Exception prepareTaskForLocking(task); Assert.assertTrue(task.isReady(actionClient)); - final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.createRunner(); + final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.createRunner(toolbox); final Iterator subTaskSpecIterator = runner.subTaskSpecIterator().iterator(); while (subTaskSpecIterator.hasNext()) { @@ -279,11 +279,12 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.fromCode( getId(), new TestRunner( + toolbox, this, indexingServiceClient, new NoopChatHandlerProvider(), new AuthorizerMapper(Collections.emptyMap()) - ).run(toolbox) + ).run() ); } } @@ -293,6 +294,7 @@ private static class TestRunner extends TestParallelIndexTaskRunner private final ParallelIndexSupervisorTask supervisorTask; TestRunner( + TaskToolbox toolbox, ParallelIndexSupervisorTask supervisorTask, @Nullable IndexingServiceClient indexingServiceClient, @Nullable ChatHandlerProvider chatHandlerProvider, @@ -300,6 +302,7 @@ private static class TestRunner extends TestParallelIndexTaskRunner ) { super( + toolbox, supervisorTask.getId(), supervisorTask.getGroupId(), supervisorTask.getIngestionSchema(), From 550fd38b67772cd4f3c0f2802ea70af2473de66f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 16 Jul 2018 21:52:07 -0700 Subject: [PATCH 38/47] Fix race in TaskMonitor and move HTTP endpoints to supervisorTask from runner --- .../task/ParallelIndexSupervisorTask.java | 344 +++++++++--- .../common/task/ParallelIndexTaskRunner.java | 69 ++- .../common/task/ParallelIndexingStatus.java | 91 ++++ .../SinglePhaseParallelIndexTaskRunner.java | 490 +++++------------- .../indexing/common/task/TaskHistory.java | 61 +++ .../indexing/common/task/TaskMonitor.java | 240 ++++----- ...stractParallelIndexSupervisorTaskTest.java | 32 +- ...rallelIndexSupervisorTaskResourceTest.java | 86 ++- .../task/ParallelIndexSupervisorTaskTest.java | 21 +- .../indexing/common/task/TaskMonitorTest.java | 1 - 10 files changed, 786 insertions(+), 649 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingStatus.java create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java index 372fccdd8196..d7bbde1c8750 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java @@ -22,30 +22,56 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import io.druid.client.indexing.IndexingServiceClient; import io.druid.data.input.FiniteFirehoseFactory; import io.druid.data.input.FirehoseFactory; import io.druid.indexer.TaskStatus; +import io.druid.indexing.common.Counters; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.stats.RowIngestionMetersFactory; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import io.druid.indexing.common.task.ParallelIndexTaskRunner.SubTaskSpecStatus; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.indexing.granularity.GranularitySpec; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.firehose.ChatHandler; import io.druid.segment.realtime.firehose.ChatHandlerProvider; +import io.druid.segment.realtime.firehose.ChatHandlers; +import io.druid.server.security.Action; import io.druid.server.security.AuthorizerMapper; +import io.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.Status; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.SortedSet; +import java.util.stream.Collectors; /** * ParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing. This is @@ -67,7 +93,12 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan private final AuthorizerMapper authorizerMapper; private final RowIngestionMetersFactory rowIngestionMetersFactory; - private ParallelIndexTaskRunner runner; + private final Counters counters = new Counters(); + + private volatile ParallelIndexTaskRunner runner; + + // toolbox is initlized when run() is called, and can be used for processing HTTP endpoint requests. + private volatile TaskToolbox toolbox; @JsonCreator public ParallelIndexSupervisorTask( @@ -156,9 +187,7 @@ ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) getGroupId(), ingestionSchema, getContext(), - indexingServiceClient, - chatHandlerProvider, - authorizerMapper + indexingServiceClient ); } return runner; @@ -192,14 +221,27 @@ static boolean isReady(TaskActionClient actionClient, SortedSet interv @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - if (baseFirehoseFactory.isSplittable()) { - return runParallel(toolbox); - } else { - log.warn( - "firehoseFactory[%s] is not splittable. Running sequentially", - baseFirehoseFactory.getClass().getSimpleName() - ); - return runSequential(toolbox); + this.toolbox = toolbox; + + log.info( + "Found chat handler of class[%s]", + Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName() + ); + chatHandlerProvider.register(getId(), this, false); + + try { + if (baseFirehoseFactory.isSplittable()) { + return runParallel(toolbox); + } else { + log.warn( + "firehoseFactory[%s] is not splittable. Running sequentially", + baseFirehoseFactory.getClass().getSimpleName() + ); + return runSequential(toolbox); + } + } + finally { + chatHandlerProvider.unregister(getId()); } } @@ -252,72 +294,248 @@ private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningC ); } - static class Status + // Internal APIs + + /** + * Allocate a new {@link SegmentIdentifier} for a request from {@link ParallelIndexSubTask}. + * The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec}) + * per bucket interval. + */ + @POST + @Path("/segment/allocate") + @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) + public Response allocateSegment( + DateTime timestamp, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck( + req, + Action.READ, + getDataSource(), + authorizerMapper + ); + + if (toolbox == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } + + try { + final SegmentIdentifier segmentIdentifier = allocateNewSegment(timestamp); + return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build(); + } + catch (IOException | IllegalStateException e) { + return Response.serverError().entity(Throwables.getStackTraceAsString(e)).build(); + } + catch (IllegalArgumentException e) { + return Response.status(Response.Status.BAD_REQUEST).entity(Throwables.getStackTraceAsString(e)).build(); + } + } + + @VisibleForTesting + SegmentIdentifier allocateNewSegment(DateTime timestamp) throws IOException + { + final String dataSource = getDataSource(); + final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec(); + final SortedSet bucketIntervals = Preconditions.checkNotNull( + granularitySpec.bucketIntervals().orNull(), + "bucketIntervals" + ); + // List locks whenever allocating a new segment because locks might be revoked and no longer valid. + final Map versions = toolbox + .getTaskActionClient() + .submit(new LockListAction()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + + final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); + if (!maybeInterval.isPresent()) { + throw new IAE("Could not find interval for timestamp [%s]", timestamp); + } + + final Interval interval = maybeInterval.get(); + if (!bucketIntervals.contains(interval)) { + throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); + } + + final int partitionNum = counters.increment(interval.toString(), 1); + return new SegmentIdentifier( + dataSource, + interval, + findVersion(versions, interval), + new NumberedShardSpec(partitionNum, 0) + ); + } + + private static String findVersion(Map versions, Interval interval) + { + return versions.entrySet().stream() + .filter(entry -> entry.getKey().contains(interval)) + .map(Entry::getValue) + .findFirst() + .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); + } + + /** + * {@link ParallelIndexSubTask}s call this API to report the segments they've generated and pushed. + */ + @POST + @Path("/report") + @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE) + public Response report( + PushedSegmentsReport report, + @Context final HttpServletRequest req + ) + { + ChatHandlers.authorizationCheck( + req, + Action.WRITE, + getDataSource(), + authorizerMapper + ); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + runner.collectReport(report); + return Response.ok().build(); + } + } + + // External APIs to get running status + + @GET + @Path("/mode") + @Produces(MediaType.APPLICATION_JSON) + public Response getMode(@Context final HttpServletRequest req) { - private final int running; - private final int succeeded; - private final int failed; - private final int complete; - private final int total; - private final int expectedSucceeded; - - static Status empty() - { - return new Status(0, 0, 0, 0, 0, 0); + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + return Response.ok(baseFirehoseFactory.isSplittable() ? "parallel" : "sequential").build(); } + } - @JsonCreator - Status( - @JsonProperty("running") int running, - @JsonProperty("succeeded") int succeeded, - @JsonProperty("failed") int failed, - @JsonProperty("complete") int complete, - @JsonProperty("total") int total, - @JsonProperty("expectedSucceeded") int expectedSucceeded - ) - { - this.running = running; - this.succeeded = succeeded; - this.failed = failed; - this.complete = complete; - this.total = total; - this.expectedSucceeded = expectedSucceeded; + @GET + @Path("/status") + @Produces(MediaType.APPLICATION_JSON) + public Response getStatus(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + return Response.ok(runner.getStatus()).build(); } + } - @JsonProperty - public int getRunning() - { - return running; + @GET + @Path("/subtasks/running") + @Produces(MediaType.APPLICATION_JSON) + public Response getRunningTasks(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + return Response.ok(runner.getRunningTaskIds()).build(); } + } - @JsonProperty - public int getSucceeded() - { - return succeeded; + @GET + @Path("/subtaskspecs") + @Produces(MediaType.APPLICATION_JSON) + public Response getSubTaskSpecs(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + return Response.ok(runner.getSubTaskSpecs()).build(); } + } - @JsonProperty - public int getFailed() - { - return failed; + @GET + @Path("/subtaskspecs/running") + @Produces(MediaType.APPLICATION_JSON) + public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + return Response.ok(runner.getRunningSubTaskSpecs()).build(); } + } - @JsonProperty - public int getComplete() - { - return complete; + @GET + @Path("/subtaskspecs/complete") + @Produces(MediaType.APPLICATION_JSON) + public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + return Response.ok(runner.getCompleteSubTaskSpecs()).build(); + } + } + + @GET + @Path("/subtaskspec/{id}") + @Produces(MediaType.APPLICATION_JSON) + public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + final SubTaskSpec subTaskSpec = runner.getSubTaskSpec(id); + if (subTaskSpec == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } else { + return Response.ok(subTaskSpec).build(); + } } + } - @JsonProperty - public int getTotal() - { - return total; + @GET + @Path("/subtaskspec/{id}/state") + @Produces(MediaType.APPLICATION_JSON) + public Response getSubTaskState(@PathParam("id") String id, @Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + final SubTaskSpecStatus subTaskSpecStatus = runner.getSubTaskState(id); + if (subTaskSpecStatus == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } else { + return Response.ok(subTaskSpecStatus).build(); + } } + } - @JsonProperty - public int getExpectedSucceeded() - { - return expectedSucceeded; + @GET + @Path("/subtaskspec/{id}/history") + @Produces(MediaType.APPLICATION_JSON) + public Response getCompleteSubTaskSpecAttemptHistory( + @PathParam("id") String id, + @Context final HttpServletRequest req + ) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); + } else { + final TaskHistory taskHistory = runner.getCompleteSubTaskSpecAttemptHistory(id); + if (taskHistory == null) { + return Response.status(Status.NOT_FOUND).build(); + } else { + return Response.ok(taskHistory.getAttemptHistory()).build(); + } } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java index 92674c5fbea4..1eb07f47db8e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java @@ -19,14 +19,81 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Set; /** * ParallelIndexTaskRunner is the actual task runner of {@link ParallelIndexSupervisorTask}. There is currently a single * implementation, i.e. {@link SinglePhaseParallelIndexTaskRunner} which supports only best-effort roll-up. We can add * more implementations in the future. */ -public interface ParallelIndexTaskRunner +public interface ParallelIndexTaskRunner { TaskState run() throws Exception; + + void collectReport(PushedSegmentsReport report); + + ParallelIndexingStatus getStatus(); + + Set getRunningTaskIds(); + + List> getSubTaskSpecs(); + + List> getRunningSubTaskSpecs(); + + List> getCompleteSubTaskSpecs(); + + @Nullable + SubTaskSpec getSubTaskSpec(String subTaskSpecId); + + @Nullable + SubTaskSpecStatus getSubTaskState(String subTaskSpecId); + + @Nullable + TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId); + + class SubTaskSpecStatus + { + private final ParallelIndexSubTaskSpec spec; + @Nullable + private final TaskStatusPlus currentStatus; + private final List taskHistory; + + @JsonCreator + public SubTaskSpecStatus( + @JsonProperty("spec") ParallelIndexSubTaskSpec spec, + @JsonProperty("currentStatus") @Nullable TaskStatusPlus currentStatus, + @JsonProperty("taskHistory") List taskHistory + ) + { + this.spec = spec; + this.currentStatus = currentStatus; + this.taskHistory = taskHistory; + } + + @JsonProperty + public ParallelIndexSubTaskSpec getSpec() + { + return spec; + } + + @JsonProperty + @Nullable + public TaskStatusPlus getCurrentStatus() + { + return currentStatus; + } + + @JsonProperty + public List getTaskHistory() + { + return taskHistory; + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingStatus.java new file mode 100644 index 000000000000..7e590e2199d7 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingStatus.java @@ -0,0 +1,91 @@ +/* + * 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 io.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +class ParallelIndexingStatus +{ + private final int running; + private final int succeeded; + private final int failed; + private final int complete; + private final int total; + private final int expectedSucceeded; + + static ParallelIndexingStatus notRunning() + { + return new ParallelIndexingStatus(0, 0, 0, 0, 0, -1); + } + + @JsonCreator + ParallelIndexingStatus( + @JsonProperty("running") int running, + @JsonProperty("succeeded") int succeeded, + @JsonProperty("failed") int failed, + @JsonProperty("complete") int complete, + @JsonProperty("total") int total, + @JsonProperty("expectedSucceeded") int expectedSucceeded + ) + { + this.running = running; + this.succeeded = succeeded; + this.failed = failed; + this.complete = complete; + this.total = total; + this.expectedSucceeded = expectedSucceeded; + } + + @JsonProperty + public int getRunning() + { + return running; + } + + @JsonProperty + public int getSucceeded() + { + return succeeded; + } + + @JsonProperty + public int getFailed() + { + return failed; + } + + @JsonProperty + public int getComplete() + { + return complete; + } + + @JsonProperty + public int getTotal() + { + return total; + } + + @JsonProperty + public int getExpectedSucceeded() + { + return expectedSucceeded; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java index db6a00171b7f..cc37c6a10066 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java @@ -19,13 +19,8 @@ package io.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -36,43 +31,18 @@ import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; -import io.druid.indexing.common.Counters; -import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status; import io.druid.indexing.common.task.TaskMonitor.MonitorEntry; import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; -import io.druid.indexing.common.task.TaskMonitor.TaskHistory; -import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; -import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.realtime.appenderator.SegmentIdentifier; import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import io.druid.segment.realtime.appenderator.UsedSegmentChecker; -import io.druid.segment.realtime.firehose.ChatHandler; -import io.druid.segment.realtime.firehose.ChatHandlerProvider; -import io.druid.segment.realtime.firehose.ChatHandlers; -import io.druid.server.security.Action; -import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.NumberedShardSpec; -import org.joda.time.DateTime; -import org.joda.time.Interval; import javax.annotation.Nullable; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -80,9 +50,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; -import java.util.SortedSet; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -98,7 +66,7 @@ * As its name indicates, distributed indexing is done in a single phase, i.e., without shuffling intermediate data. As * a result, this task can't be used for perfect rollup. */ -public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunner, ChatHandler +public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunner { private static final Logger log = new Logger(SinglePhaseParallelIndexTaskRunner.class); @@ -110,8 +78,6 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn private final FiniteFirehoseFactory baseFirehoseFactory; private final int maxNumTasks; private final IndexingServiceClient indexingServiceClient; - private final ChatHandlerProvider chatHandlerProvider; - private final AuthorizerMapper authorizerMapper; private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); @@ -119,8 +85,6 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn // subTaskId -> report private final ConcurrentMap segmentsMap = new ConcurrentHashMap<>(); - private final Counters counters = new Counters(); - private volatile boolean stopped; private volatile TaskMonitor taskMonitor; @@ -132,9 +96,7 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn String groupId, ParallelIndexIngestionSpec ingestionSchema, Map context, - IndexingServiceClient indexingServiceClient, - ChatHandlerProvider chatHandlerProvider, - AuthorizerMapper authorizerMapper + IndexingServiceClient indexingServiceClient ) { this.toolbox = toolbox; @@ -145,19 +107,11 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory(); this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumSubTasks(); this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"); - this.chatHandlerProvider = Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider"); - this.authorizerMapper = authorizerMapper; } @Override public TaskState run() throws Exception { - log.info( - "Found chat handler of class[%s]", - Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName() - ); - chatHandlerProvider.register(taskId, this, false); - final Iterator subTaskSpecIterator = subTaskSpecIterator().iterator(); final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs(); @@ -173,12 +127,12 @@ public TaskState run() throws Exception try { log.info("Submitting initial tasks"); // Submit initial tasks - while (subTaskSpecIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) { + while (isRunning() && subTaskSpecIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) { submitNewTask(taskMonitor, subTaskSpecIterator.next()); } log.info("Waiting for subTasks to be completed"); - while (!stopped && !Thread.currentThread().isInterrupted()) { + while (isRunning()) { final SubTaskCompleteEvent taskCompleteEvent = taskCompleteEvents.poll( taskStatusCheckingPeriod, TimeUnit.MILLISECONDS @@ -209,7 +163,7 @@ public TaskState run() throws Exception state = TaskState.SUCCESS; } else { // Failed - final Status monitorStatus = taskMonitor.getStatus(); + final ParallelIndexingStatus monitorStatus = taskMonitor.getStatus(); throw new ISE( "Expected for [%d] tasks to succeed, but we got [%d] succeeded tasks and [%d] failed tasks", monitorStatus.getExpectedSucceeded(), @@ -253,7 +207,6 @@ public TaskState run() throws Exception // Cleanup resources taskCompleteEvents.clear(); taskMonitor.stop(); - chatHandlerProvider.unregister(taskId); if (state != TaskState.SUCCESS) { log.info( @@ -271,6 +224,11 @@ public TaskState run() throws Exception return state; } + private boolean isRunning() + { + return !stopped && !Thread.currentThread().isInterrupted(); + } + @VisibleForTesting TaskToolbox getToolbox() { @@ -283,127 +241,141 @@ ParallelIndexIngestionSpec getIngestionSchema() return ingestionSchema; } - private String getDataSource() + @Override + public void collectReport(PushedSegmentsReport report) { - return ingestionSchema.getDataSchema().getDataSource(); + // subTasks might send their reports multiple times because of the HTTP retry. + // Here, we simply make sure the current report is exactly same with the previous one. + segmentsMap.compute(report.getTaskId(), (taskId, prevReport) -> { + if (prevReport != null) { + Preconditions.checkState( + prevReport.getSegments().equals(report.getSegments()), + "task[%s] sent two or more reports and previous report[%s] is different from the current one[%s]", + taskId, + prevReport, + report + ); + } + return report; + }); } - // Internal APIs - - /** - * Allocate a new {@link SegmentIdentifier} for a request from {@link ParallelIndexSubTask}. - * The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec}) - * per bucket interval. - */ - @POST - @Path("/segment/allocate") - @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) - public Response allocateSegment( - DateTime timestamp, - @Context final HttpServletRequest req - ) + @Override + public ParallelIndexingStatus getStatus() { - ChatHandlers.authorizationCheck( - req, - Action.READ, - getDataSource(), - authorizerMapper - ); - - try { - final SegmentIdentifier segmentIdentifier = allocateNewSegment(timestamp); - return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build(); - } - catch (IOException | IllegalStateException e) { - return Response.serverError().entity(Throwables.getStackTraceAsString(e)).build(); - } - catch (IllegalArgumentException e) { - return Response.status(Response.Status.BAD_REQUEST).entity(Throwables.getStackTraceAsString(e)).build(); - } + return taskMonitor == null ? ParallelIndexingStatus.notRunning() : taskMonitor.getStatus(); } - @VisibleForTesting - SegmentIdentifier allocateNewSegment(DateTime timestamp) throws IOException + @Override + public Set getRunningTaskIds() { - final String dataSource = getDataSource(); - final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec(); - final SortedSet bucketIntervals = Preconditions.checkNotNull( - granularitySpec.bucketIntervals().orNull(), - "bucketIntervals" - ); - // List locks whenever allocating a new segment because locks might be revoked and no longer valid. - final Map versions = toolbox - .getTaskActionClient() - .submit(new LockListAction()) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - - final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); - if (!maybeInterval.isPresent()) { - throw new IAE("Could not find interval for timestamp [%s]", timestamp); - } + return taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds(); + } - final Interval interval = maybeInterval.get(); - if (!bucketIntervals.contains(interval)) { - throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); + @Override + public List> getSubTaskSpecs() + { + if (taskMonitor != null) { + final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); + final List> completeSubTaskSpecs = taskMonitor + .getCompleteSubTaskSpecs(); + // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and + // completeSubTaskSpecs. + final Map> subTaskSpecMap = new HashMap<>( + runningSubTaskSpecs.size() + completeSubTaskSpecs.size() + ); + runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); + completeSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); + return new ArrayList<>(subTaskSpecMap.values()); + } else { + return Collections.emptyList(); } + } - final int partitionNum = counters.increment(interval.toString(), 1); - return new SegmentIdentifier( - dataSource, - interval, - findVersion(versions, interval), - new NumberedShardSpec(partitionNum, 0) - ); + @Override + public List> getRunningSubTaskSpecs() + { + return taskMonitor == null ? Collections.emptyList() : taskMonitor.getRunningSubTaskSpecs(); } - private static String findVersion(Map versions, Interval interval) + @Override + public List> getCompleteSubTaskSpecs() { - return versions.entrySet().stream() - .filter(entry -> entry.getKey().contains(interval)) - .map(Entry::getValue) - .findFirst() - .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); + return taskMonitor == null ? Collections.emptyList() : taskMonitor.getCompleteSubTaskSpecs(); } - /** - * {@link ParallelIndexSubTask}s call this API to report the segments they've generated and pushed. - */ - @POST - @Path("/report") - @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE) - public Response report( - PushedSegmentsReport report, - @Context final HttpServletRequest req - ) + @Nullable + @Override + public SubTaskSpec getSubTaskSpec(String subTaskSpecId) { - ChatHandlers.authorizationCheck( - req, - Action.WRITE, - getDataSource(), - authorizerMapper - ); - collectReport(report); - return Response.ok().build(); + if (taskMonitor != null) { + // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after + // adding them to taskHistory. + final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(subTaskSpecId); + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); + final SubTaskSpec subTaskSpec; + + if (monitorEntry != null) { + subTaskSpec = monitorEntry.getSpec(); + } else { + if (taskHistory != null) { + subTaskSpec = taskHistory.getSpec(); + } else { + subTaskSpec = null; + } + } + + return subTaskSpec; + } else { + return null; + } } - @VisibleForTesting - void collectReport(PushedSegmentsReport report) + @Nullable + @Override + public SubTaskSpecStatus getSubTaskState(String subTaskSpecId) { - // subTasks might send their reports multiple times because of the HTTP retry. - // Here, we simply make sure the current report is exactly same with the previous one. - segmentsMap.compute(report.getTaskId(), (taskId, prevReport) -> { - if (prevReport != null) { - Preconditions.checkState( - prevReport.getSegments().equals(report.getSegments()), - "task[%s] sent two or more reports and previous report[%s] is different from the current one[%s]", - taskId, - prevReport, - report + if (taskMonitor == null) { + return null; + } else { + // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after + // adding them to taskHistory. + final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(subTaskSpecId); + final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); + + final SubTaskSpecStatus subTaskSpecStatus; + + if (monitorEntry != null) { + subTaskSpecStatus = new SubTaskSpecStatus( + (ParallelIndexSubTaskSpec) monitorEntry.getSpec(), + monitorEntry.getRunningStatus(), + monitorEntry.getTaskHistory() ); + } else { + if (taskHistory != null && !taskHistory.isEmpty()) { + subTaskSpecStatus = new SubTaskSpecStatus( + (ParallelIndexSubTaskSpec) taskHistory.getSpec(), + null, + taskHistory.getAttemptHistory() + ); + } else { + subTaskSpecStatus = null; + } } - return report; - }); + + return subTaskSpecStatus; + } + } + + @Nullable + @Override + public TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId) + { + if (taskMonitor == null) { + return null; + } else { + return taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); + } } private void publish(TaskToolbox toolbox) throws IOException @@ -509,218 +481,4 @@ private static List getSplitsIfSplittable(FirehoseFactory firehoseFa throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName()); } } - - // External APIs to get running status - - @GET - @Path("/mode") - @Produces(MediaType.APPLICATION_JSON) - public Response isRunningInParallel(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - return Response.ok(baseFirehoseFactory.isSplittable() ? "parallel" : "sequential").build(); - } - - @GET - @Path("/status") - @Produces(MediaType.APPLICATION_JSON) - public Response getStatus(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - return Response.ok(taskMonitor == null ? Status.empty() : taskMonitor.getStatus()).build(); - } - - @GET - @Path("/subtasks/running") - @Produces(MediaType.APPLICATION_JSON) - public Response getRunningTasks(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final Set runningTasks = taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds(); - return Response.ok(runningTasks).build(); - } - - @GET - @Path("/subtaskspecs") - @Produces(MediaType.APPLICATION_JSON) - public Response getSubTaskSpecs(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (taskMonitor != null) { - final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); - final List> completeSubTaskSpecs = taskMonitor - .getCompleteSubTaskSpecs(); - // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and - // completeSubTaskSpecs. - final Map> subTaskSpecMap = new HashMap<>( - runningSubTaskSpecs.size() + completeSubTaskSpecs.size() - ); - runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); - completeSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec)); - return Response.ok(new ArrayList<>(subTaskSpecMap.values())).build(); - } else { - return Response.ok(Collections.emptyList()).build(); - } - } - - @GET - @Path("/subtaskspecs/running") - @Produces(MediaType.APPLICATION_JSON) - public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final List> runningSubTaskSpecs = taskMonitor == null ? - Collections.emptyList() : - taskMonitor.getRunningSubTaskSpecs(); - return Response.ok(runningSubTaskSpecs).build(); - } - - @GET - @Path("/subtaskspecs/complete") - @Produces(MediaType.APPLICATION_JSON) - public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - final List> completeSubTaskSpecs = - taskMonitor == null ? - Collections.emptyList() : - taskMonitor.getCompleteSubTaskSpecs(); - return Response.ok(completeSubTaskSpecs).build(); - } - - @GET - @Path("/subtaskspec/{id}") - @Produces(MediaType.APPLICATION_JSON) - public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (taskMonitor != null) { - // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after - // adding them to taskHistory. - final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); - final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); - final SubTaskSpec subTaskSpec; - - if (monitorEntry != null) { - subTaskSpec = monitorEntry.getSpec(); - } else { - if (taskHistory != null) { - subTaskSpec = taskHistory.getSpec(); - } else { - subTaskSpec = null; - } - } - - if (subTaskSpec == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } else { - return Response.ok(subTaskSpec).build(); - } - } else { - return Response.status(Response.Status.NOT_FOUND).build(); - } - } - - @GET - @Path("/subtaskspec/{id}/state") - @Produces(MediaType.APPLICATION_JSON) - public Response getSubTaskState(@PathParam("id") String id, @Context final HttpServletRequest req) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (taskMonitor == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } else { - // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after - // adding them to taskHistory. - final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(id); - final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); - - final SubTaskStateResponse subTaskStateResponse; - - if (monitorEntry != null) { - subTaskStateResponse = new SubTaskStateResponse( - (ParallelIndexSubTaskSpec) monitorEntry.getSpec(), - monitorEntry.getRunningStatus(), - monitorEntry.getTaskHistory() - ); - } else { - if (taskHistory != null && !taskHistory.isEmpty()) { - subTaskStateResponse = new SubTaskStateResponse( - (ParallelIndexSubTaskSpec) taskHistory.getSpec(), - null, - taskHistory.getAttemptHistory() - ); - } else { - subTaskStateResponse = null; - } - } - - if (subTaskStateResponse == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } else { - return Response.ok(subTaskStateResponse).build(); - } - } - } - - @GET - @Path("/subtaskspec/{id}/history") - @Produces(MediaType.APPLICATION_JSON) - public Response getCompleteSubTaskSpecAttemptHistory( - @PathParam("id") String id, - @Context final HttpServletRequest req - ) - { - IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (taskMonitor == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } else { - final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(id); - - if (taskHistory == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } else { - return Response.ok(taskHistory.getAttemptHistory()).build(); - } - } - } - - static class SubTaskStateResponse - { - private final ParallelIndexSubTaskSpec spec; - @Nullable - private final TaskStatusPlus currentStatus; - private final List taskHistory; - - @JsonCreator - public SubTaskStateResponse( - @JsonProperty("spec") ParallelIndexSubTaskSpec spec, - @JsonProperty("currentStatus") @Nullable TaskStatusPlus currentStatus, - @JsonProperty("taskHistory") List taskHistory - ) - { - this.spec = spec; - this.currentStatus = currentStatus; - this.taskHistory = taskHistory; - } - - @JsonProperty - public ParallelIndexSubTaskSpec getSpec() - { - return spec; - } - - @JsonProperty - @Nullable - public TaskStatusPlus getCurrentStatus() - { - return currentStatus; - } - - @JsonProperty - public List getTaskHistory() - { - return taskHistory; - } - } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java new file mode 100644 index 000000000000..b8710c27e525 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java @@ -0,0 +1,61 @@ +/* + * 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 io.druid.indexing.common.task; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import io.druid.indexer.TaskState; +import io.druid.indexer.TaskStatusPlus; + +import java.util.List; + +class TaskHistory +{ + private final SubTaskSpec spec; + private final List attemptHistory; // old to recent + + TaskHistory(SubTaskSpec spec, List attemptHistory) + { + attemptHistory.forEach(status -> { + Preconditions.checkState( + status.getState() == TaskState.SUCCESS || status.getState() == TaskState.FAILED, + "Complete tasks should be recorded, but the state of task[%s] is [%s]", + status.getId(), + status.getState() + ); + }); + this.spec = spec; + this.attemptHistory = ImmutableList.copyOf(attemptHistory); + } + + SubTaskSpec getSpec() + { + return spec; + } + + List getAttemptHistory() + { + return attemptHistory; + } + + boolean isEmpty() + { + return attemptHistory.isEmpty(); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index b7fabde5f0ae..b0d70223cf6c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -20,7 +20,6 @@ package io.druid.indexing.common.task; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; @@ -28,7 +27,6 @@ import io.druid.client.indexing.TaskStatusResponse; import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; -import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; @@ -74,6 +72,9 @@ public class TaskMonitor // lock for updating numRunningTasks, numSucceededTasks, and numFailedTasks private final Object taskCountLock = new Object(); + // lock for updating running state + private final Object startStopLock = new Object(); + // overlord client private final IndexingServiceClient indexingServiceClient; private final int maxRetry; @@ -83,7 +84,7 @@ public class TaskMonitor private int numSucceededTasks; private int numFailedTasks; - private volatile boolean running = false; + private boolean running = false; TaskMonitor(IndexingServiceClient indexingServiceClient, int maxRetry, int expectedNumSucceededTasks) { @@ -96,99 +97,105 @@ public class TaskMonitor public void start(long taskStatusCheckingPeriod) { - running = true; - log.info("Starting taskMonitor"); - // NOTE: This polling can be improved to event-driven pushing by registering TaskRunnerListener to TaskRunner. - // That listener should be able to send the events reported to TaskRunner to this TaskMonitor. - taskStatusChecker.scheduleAtFixedRate( - () -> { - try { - final Iterator> iterator = runningTasks.entrySet().iterator(); - while (iterator.hasNext()) { - final Entry entry = iterator.next(); - final String specId = entry.getKey(); - final MonitorEntry monitorEntry = entry.getValue(); - final String taskId = monitorEntry.runningTask.getId(); - final TaskStatusResponse taskStatusResponse = indexingServiceClient.getTaskStatus(taskId); - final TaskStatusPlus taskStatus = taskStatusResponse.getStatus(); - if (taskStatus != null) { - switch (Preconditions.checkNotNull(taskStatus.getState(), "taskState")) { - case SUCCESS: - incrementNumSucceededTasks(); - - // Remote the current entry after updating taskHistories to make sure that task history - // exists either runningTasks or taskHistories. - monitorEntry.setLastStatus(taskStatus); - iterator.remove(); - break; - case FAILED: - incrementNumFailedTasks(); - - log.warn("task[%s] failed!", taskId); - if (monitorEntry.numTries() < maxRetry) { - log.info( - "We still have chances[%d/%d] to complete for spec[%s].", - monitorEntry.numTries(), - maxRetry, - monitorEntry.spec.getId() - ); - retry(specId, monitorEntry, taskStatus); - } else { - log.error( - "spec[%s] failed after [%d] tries", - monitorEntry.spec.getId(), - monitorEntry.numTries() - ); + synchronized (startStopLock) { + running = true; + log.info("Starting taskMonitor"); + // NOTE: This polling can be improved to event-driven pushing by registering TaskRunnerListener to TaskRunner. + // That listener should be able to send the events reported to TaskRunner to this TaskMonitor. + taskStatusChecker.scheduleAtFixedRate( + () -> { + try { + final Iterator> iterator = runningTasks.entrySet().iterator(); + while (iterator.hasNext()) { + final Entry entry = iterator.next(); + final String specId = entry.getKey(); + final MonitorEntry monitorEntry = entry.getValue(); + final String taskId = monitorEntry.runningTask.getId(); + final TaskStatusResponse taskStatusResponse = indexingServiceClient.getTaskStatus(taskId); + final TaskStatusPlus taskStatus = taskStatusResponse.getStatus(); + if (taskStatus != null) { + switch (Preconditions.checkNotNull(taskStatus.getState(), "taskState")) { + case SUCCESS: + incrementNumSucceededTasks(); + // Remote the current entry after updating taskHistories to make sure that task history // exists either runningTasks or taskHistories. monitorEntry.setLastStatus(taskStatus); iterator.remove(); - } - break; - case RUNNING: - monitorEntry.updateStatus(taskStatus); - break; - default: - throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getState(), taskId); + break; + case FAILED: + incrementNumFailedTasks(); + + log.warn("task[%s] failed!", taskId); + if (monitorEntry.numTries() < maxRetry) { + log.info( + "We still have chances[%d/%d] to complete for spec[%s].", + monitorEntry.numTries(), + maxRetry, + monitorEntry.spec.getId() + ); + retry(specId, monitorEntry, taskStatus); + } else { + log.error( + "spec[%s] failed after [%d] tries", + monitorEntry.spec.getId(), + monitorEntry.numTries() + ); + // Remote the current entry after updating taskHistories to make sure that task history + // exists either runningTasks or taskHistories. + monitorEntry.setLastStatus(taskStatus); + iterator.remove(); + } + break; + case RUNNING: + monitorEntry.updateStatus(taskStatus); + break; + default: + throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getState(), taskId); + } } } } - } - catch (Throwable t) { - log.error(t, "Error while monitoring"); - throw t; - } - }, - taskStatusCheckingPeriod, - taskStatusCheckingPeriod, - TimeUnit.MILLISECONDS - ); + catch (Throwable t) { + log.error(t, "Error while monitoring"); + throw t; + } + }, + taskStatusCheckingPeriod, + taskStatusCheckingPeriod, + TimeUnit.MILLISECONDS + ); + } } public void stop() { - running = false; - taskStatusChecker.shutdownNow(); - log.info("Stopped taskMonitor"); + synchronized (startStopLock) { + running = false; + taskStatusChecker.shutdownNow(); + log.info("Stopped taskMonitor"); + } } public ListenableFuture> submit(SubTaskSpec spec) { - if (!running) { - return Futures.immediateFailedFuture(new ISE("TaskMonitore is not running")); + synchronized (startStopLock) { + if (!running) { + return Futures.immediateFailedFuture(new ISE("TaskMonitore is not running")); + } + final T task = spec.newSubTask(0); + log.info("Submitting a new task[%s] for spec[%s]", task.getId(), spec.getId()); + indexingServiceClient.runTask(task); + incrementNumRunningTasks(); + + final SettableFuture> taskFuture = SettableFuture.create(); + runningTasks.put( + spec.getId(), + new MonitorEntry(spec, task, indexingServiceClient.getTaskStatus(task.getId()).getStatus(), taskFuture) + ); + + return taskFuture; } - final T task = spec.newSubTask(0); - log.info("Submitting a new task[%s] for spec[%s]", task.getId(), spec.getId()); - indexingServiceClient.runTask(task); - incrementNumRunningTasks(); - - final SettableFuture> taskFuture = SettableFuture.create(); - runningTasks.put( - spec.getId(), - new MonitorEntry(spec, task, indexingServiceClient.getTaskStatus(task.getId()).getStatus(), taskFuture) - ); - - return taskFuture; } /** @@ -197,21 +204,23 @@ public ListenableFuture> submit(SubTaskSpec spec) */ private void retry(String subTaskSpecId, MonitorEntry monitorEntry, TaskStatusPlus lastFailedTaskStatus) { - if (running) { - final SubTaskSpec spec = monitorEntry.spec; - final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1); - log.info("Submitting a new task[%s] for retrying spec[%s]", task.getId(), spec.getId()); - indexingServiceClient.runTask(task); - incrementNumRunningTasks(); - - runningTasks.put( - subTaskSpecId, - monitorEntry.withNewRunningTask( - task, - indexingServiceClient.getTaskStatus(task.getId()).getStatus(), - lastFailedTaskStatus - ) - ); + synchronized (startStopLock) { + if (running) { + final SubTaskSpec spec = monitorEntry.spec; + final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1); + log.info("Submitting a new task[%s] for retrying spec[%s]", task.getId(), spec.getId()); + indexingServiceClient.runTask(task); + incrementNumRunningTasks(); + + runningTasks.put( + subTaskSpecId, + monitorEntry.withNewRunningTask( + task, + indexingServiceClient.getTaskStatus(task.getId()).getStatus(), + lastFailedTaskStatus + ) + ); + } } } @@ -266,10 +275,10 @@ int getNumRunningTasks() } } - Status getStatus() + ParallelIndexingStatus getStatus() { synchronized (taskCountLock) { - return new Status( + return new ParallelIndexingStatus( numRunningTasks, numSucceededTasks, numFailedTasks, @@ -302,7 +311,7 @@ MonitorEntry getRunningTaskMonitorEntory(String subTaskSpecId) List> getCompleteSubTaskSpecs() { - return taskHistories.values().stream().map(history -> history.spec).collect(Collectors.toList()); + return taskHistories.values().stream().map(TaskHistory::getSpec).collect(Collectors.toList()); } @Nullable @@ -413,41 +422,6 @@ List getTaskHistory() } } - static class TaskHistory - { - private final SubTaskSpec spec; - private final List attemptHistory; // old to recent - - TaskHistory(SubTaskSpec spec, List attemptHistory) - { - attemptHistory.forEach(status -> { - Preconditions.checkState( - status.getState() == TaskState.SUCCESS || status.getState() == TaskState.FAILED, - "Complete tasks should be recorded, but the state of task[%s] is [%s]", - status.getId(), - status.getState() - ); - }); - this.spec = spec; - this.attemptHistory = ImmutableList.copyOf(attemptHistory); - } - - SubTaskSpec getSpec() - { - return spec; - } - - List getAttemptHistory() - { - return attemptHistory; - } - - boolean isEmpty() - { - return attemptHistory.isEmpty(); - } - } - static class SubTaskCompleteEvent { private final SubTaskSpec spec; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java index 029289d74fce..d789440b9bdd 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java @@ -49,7 +49,6 @@ import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.segment.realtime.appenderator.SegmentIdentifier; -import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.Authorizer; @@ -64,7 +63,6 @@ import java.io.File; import java.io.IOException; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -303,9 +301,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception getGroupId(), getIngestionSchema(), getContext(), - new NoopIndexingServiceClient(), - new NoopChatHandlerProvider(), - new AuthorizerMapper(Collections.emptyMap()) + new NoopIndexingServiceClient() ).run() ); } @@ -319,9 +315,7 @@ static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRun String groupId, ParallelIndexIngestionSpec ingestionSchema, Map context, - @Nullable IndexingServiceClient indexingServiceClient, - @Nullable ChatHandlerProvider chatHandlerProvider, - AuthorizerMapper authorizerMapper + @Nullable IndexingServiceClient indexingServiceClient ) { super( @@ -330,9 +324,7 @@ static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRun groupId, ingestionSchema, context, - indexingServiceClient, - chatHandlerProvider, - authorizerMapper + indexingServiceClient ); } @@ -357,11 +349,11 @@ Stream subTaskSpecIterator() throws IOException static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory { - private final SinglePhaseParallelIndexTaskRunner runner; + private final ParallelIndexSupervisorTask supervisorTask; - LocalParallelIndexTaskClientFactory(SinglePhaseParallelIndexTaskRunner runner) + LocalParallelIndexTaskClientFactory(ParallelIndexSupervisorTask supervisorTask) { - this.runner = runner; + this.supervisorTask = supervisorTask; } @Override @@ -373,30 +365,30 @@ public ParallelIndexTaskClient build( long numRetries ) { - return new LocalParallelIndexTaskClient(callerId, runner); + return new LocalParallelIndexTaskClient(callerId, supervisorTask); } } static class LocalParallelIndexTaskClient extends ParallelIndexTaskClient { - private final SinglePhaseParallelIndexTaskRunner runner; + private final ParallelIndexSupervisorTask supervisorTask; - LocalParallelIndexTaskClient(String callerId, SinglePhaseParallelIndexTaskRunner runner) + LocalParallelIndexTaskClient(String callerId, ParallelIndexSupervisorTask supervisorTask) { super(null, null, null, null, callerId, 0); - this.runner = runner; + this.supervisorTask = supervisorTask; } @Override public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException { - return runner.allocateNewSegment(timestamp); + return supervisorTask.allocateNewSegment(timestamp); } @Override public void report(String supervisorTaskId, List pushedSegments) { - runner.collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); + supervisorTask.getRunner().collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java index 4df477807dae..dddb8bbe1faf 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java @@ -35,8 +35,7 @@ import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.common.task.ParallelIndexSupervisorTask.Status; -import io.druid.indexing.common.task.SinglePhaseParallelIndexTaskRunner.SubTaskStateResponse; +import io.druid.indexing.common.task.ParallelIndexTaskRunner.SubTaskSpecStatus; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; @@ -46,11 +45,8 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; -import io.druid.segment.realtime.firehose.ChatHandlerProvider; -import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationResult; -import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; @@ -143,19 +139,19 @@ public void testAPIs() throws Exception runner = (SinglePhaseParallelIndexTaskRunner) task.getRunner(); Assert.assertNotNull("runner is null", runner); - // test isRunningInParallel - Response response = runner.isRunningInParallel(newRequest()); + // test getMode + Response response = task.getMode(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals("parallel", response.getEntity()); // test expectedNumSucceededTasks - response = runner.getStatus(newRequest()); + response = task.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(NUM_SUB_TASKS, ((Status) response.getEntity()).getExpectedSucceeded()); + Assert.assertEquals(NUM_SUB_TASKS, ((ParallelIndexingStatus) response.getEntity()).getExpectedSucceeded()); // Since taskMonitor works based on polling, it's hard to use a fancier way to check its state. // We use polling to check the state of taskMonitor in this test. - while (getNumSubTasks(Status::getRunning) < NUM_SUB_TASKS) { + while (getNumSubTasks(ParallelIndexingStatus::getRunning) < NUM_SUB_TASKS) { Thread.sleep(100); } @@ -173,7 +169,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks(Status::getSucceeded) < succeededTasks) { + while (getNumSubTasks(ParallelIndexingStatus::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -190,7 +186,7 @@ public void testAPIs() throws Exception } // Wait for new tasks to be started - while (getNumSubTasks(Status::getFailed) < failedTasks || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { + while (getNumSubTasks(ParallelIndexingStatus::getFailed) < failedTasks || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { Thread.sleep(100); } @@ -206,7 +202,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks(Status::getSucceeded) < succeededTasks) { + while (getNumSubTasks(ParallelIndexingStatus::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -225,10 +221,10 @@ public void testAPIs() throws Exception // Test one more failure runningTasks.get(0).setState(TaskState.FAILED); failedTasks++; - while (getNumSubTasks(Status::getFailed) < failedTasks) { + while (getNumSubTasks(ParallelIndexingStatus::getFailed) < failedTasks) { Thread.sleep(100); } - while (getNumSubTasks(Status::getRunning) < 1) { + while (getNumSubTasks(ParallelIndexingStatus::getRunning) < 1) { Thread.sleep(100); } @@ -241,7 +237,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); succeededTasks++; - while (getNumSubTasks(Status::getSucceeded) < succeededTasks) { + while (getNumSubTasks(ParallelIndexingStatus::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -249,22 +245,22 @@ public void testAPIs() throws Exception } @SuppressWarnings({"ConstantConditions"}) - private int getNumSubTasks(Function func) + private int getNumSubTasks(Function func) { - final Response response = runner.getStatus(newRequest()); + final Response response = task.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); - return func.apply((Status) response.getEntity()); + return func.apply((ParallelIndexingStatus) response.getEntity()); } - private Map buildStateMap() + private Map buildStateMap() { - final Map stateMap = new HashMap<>(); + final Map stateMap = new HashMap<>(); subTaskSpecs.forEach((specId, spec) -> { final List taskHistory = taskHistories.get(specId); final TaskStatusPlus runningTaskStatus = runningSpecs.get(specId); stateMap.put( specId, - new SubTaskStateResponse(spec, runningTaskStatus, taskHistory == null ? Collections.emptyList() : taskHistory) + new SubTaskSpecStatus(spec, runningTaskStatus, taskHistory == null ? Collections.emptyList() : taskHistory) ); }); return stateMap; @@ -276,12 +272,12 @@ private Map buildStateMap() private void checkState( int expectedSucceededTasks, int expectedFailedTask, - Map expectedSubTaskStateResponses // subTaskSpecId -> response + Map expectedSubTaskStateResponses // subTaskSpecId -> response ) { - Response response = runner.getStatus(newRequest()); + Response response = task.getStatus(newRequest()); Assert.assertEquals(200, response.getStatus()); - final Status monitorStatus = (Status) response.getEntity(); + final ParallelIndexingStatus monitorStatus = (ParallelIndexingStatus) response.getEntity(); // numRunningTasks Assert.assertEquals(runningTasks.size(), monitorStatus.getRunning()); @@ -299,7 +295,7 @@ private void checkState( Assert.assertEquals(runningTasks.size() + expectedSucceededTasks + expectedFailedTask, monitorStatus.getTotal()); // runningSubTasks - response = runner.getRunningTasks(newRequest()); + response = task.getRunningTasks(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals( runningTasks.stream().map(AbstractTask::getId).collect(Collectors.toSet()), @@ -307,7 +303,7 @@ private void checkState( ); // subTaskSpecs - response = runner.getSubTaskSpecs(newRequest()); + response = task.getSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); List> actualSubTaskSpecMap = (List>) response.getEntity(); @@ -317,7 +313,7 @@ private void checkState( ); // runningSubTaskSpecs - response = runner.getRunningSubTaskSpecs(newRequest()); + response = task.getRunningSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); actualSubTaskSpecMap = (List>) response.getEntity(); @@ -334,27 +330,27 @@ private void checkState( .map(entry -> entry.getValue().getSpec()) .collect(Collectors.toList()); - response = runner.getCompleteSubTaskSpecs(newRequest()); + response = task.getCompleteSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(completeSubTaskSpecs, response.getEntity()); // subTaskSpec final String subTaskId = runningSpecs.keySet().iterator().next(); - response = runner.getSubTaskSpec(subTaskId, newRequest()); + response = task.getSubTaskSpec(subTaskId, newRequest()); Assert.assertEquals(200, response.getStatus()); final SubTaskSpec subTaskSpec = (SubTaskSpec) response.getEntity(); Assert.assertEquals(subTaskId, subTaskSpec.getId()); // subTaskState - response = runner.getSubTaskState(subTaskId, newRequest()); + response = task.getSubTaskState(subTaskId, newRequest()); Assert.assertEquals(200, response.getStatus()); - final SubTaskStateResponse expectedResponse = Preconditions.checkNotNull( + final SubTaskSpecStatus expectedResponse = Preconditions.checkNotNull( expectedSubTaskStateResponses.get(subTaskId), "response for task[%s]", subTaskId ); - final SubTaskStateResponse actualResponse = (SubTaskStateResponse) response.getEntity(); + final SubTaskSpecStatus actualResponse = (SubTaskSpecStatus) response.getEntity(); Assert.assertEquals(expectedResponse.getSpec().getId(), actualResponse.getSpec().getId()); Assert.assertEquals(expectedResponse.getCurrentStatus(), actualResponse.getCurrentStatus()); Assert.assertEquals(expectedResponse.getTaskHistory(), actualResponse.getTaskHistory()); @@ -372,7 +368,7 @@ private void checkState( .findFirst() .orElse(null); if (completeSubTaskSpecId != null) { - response = runner.getCompleteSubTaskSpecAttemptHistory(completeSubTaskSpecId, newRequest()); + response = task.getCompleteSubTaskSpecAttemptHistory(completeSubTaskSpecId, newRequest()); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals( expectedSubTaskStateResponses.get(completeSubTaskSpecId).getTaskHistory(), @@ -508,9 +504,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception this.runner = new TestRunner( toolbox, this, - indexingServiceClient, - new NoopChatHandlerProvider(), - getAuthorizerMapper() + indexingServiceClient ); return TaskStatus.fromCode( getId(), @@ -532,9 +526,7 @@ private class TestRunner extends TestParallelIndexTaskRunner TestRunner( TaskToolbox toolbox, ParallelIndexSupervisorTask supervisorTask, - @Nullable IndexingServiceClient indexingServiceClient, - @Nullable ChatHandlerProvider chatHandlerProvider, - AuthorizerMapper authorizerMapper + @Nullable IndexingServiceClient indexingServiceClient ) { super( @@ -543,9 +535,7 @@ private class TestRunner extends TestParallelIndexTaskRunner supervisorTask.getGroupId(), supervisorTask.getIngestionSchema(), supervisorTask.getContext(), - indexingServiceClient, - chatHandlerProvider, - authorizerMapper + indexingServiceClient ); this.supervisorTask = supervisorTask; } @@ -557,7 +547,6 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) .getIOConfig() .getFirehoseFactory(); final TestSubTaskSpec spec = new TestSubTaskSpec( - toolbox, supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), supervisorTask, @@ -580,11 +569,9 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) private class TestSubTaskSpec extends ParallelIndexSubTaskSpec { - private final TaskToolbox toolbox; - private final SinglePhaseParallelIndexTaskRunner runner; + private final ParallelIndexSupervisorTask supervisorTask; TestSubTaskSpec( - TaskToolbox toolbox, String id, String groupId, ParallelIndexSupervisorTask supervisorTask, @@ -595,8 +582,7 @@ private class TestSubTaskSpec extends ParallelIndexSubTaskSpec ) { super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit); - this.toolbox = toolbox; - this.runner = runner; + this.supervisorTask = supervisorTask; } @Override @@ -615,7 +601,7 @@ public ParallelIndexSubTask newSubTask(int numAttempts) numAttempts, getIngestionSpec(), getContext(), - new LocalParallelIndexTaskClientFactory(runner) + new LocalParallelIndexTaskClientFactory(supervisorTask) ); final TestFirehose firehose = (TestFirehose) getIngestionSpec().getIOConfig().getFirehoseFactory(); final InputSplit split = firehose.getSplits().findFirst().orElse(null); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java index cd845bef1131..8cbfba110272 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java @@ -34,10 +34,7 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; -import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; -import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; -import io.druid.server.security.AuthorizerMapper; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -281,9 +278,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception new TestRunner( toolbox, this, - indexingServiceClient, - new NoopChatHandlerProvider(), - new AuthorizerMapper(Collections.emptyMap()) + indexingServiceClient ).run() ); } @@ -296,9 +291,7 @@ private static class TestRunner extends TestParallelIndexTaskRunner TestRunner( TaskToolbox toolbox, ParallelIndexSupervisorTask supervisorTask, - @Nullable IndexingServiceClient indexingServiceClient, - @Nullable ChatHandlerProvider chatHandlerProvider, - AuthorizerMapper authorizerMapper + @Nullable IndexingServiceClient indexingServiceClient ) { super( @@ -307,9 +300,7 @@ private static class TestRunner extends TestParallelIndexTaskRunner supervisorTask.getGroupId(), supervisorTask.getIngestionSchema(), supervisorTask.getContext(), - indexingServiceClient, - chatHandlerProvider, - authorizerMapper + indexingServiceClient ); this.supervisorTask = supervisorTask; } @@ -341,7 +332,7 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSpec { - private final SinglePhaseParallelIndexTaskRunner runner; + private final ParallelIndexSupervisorTask supervisorTask; TestParallelIndexSubTaskSpec( String id, @@ -354,7 +345,7 @@ private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSp ) { super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit); - this.runner = runner; + this.supervisorTask = supervisorTask; } @Override @@ -369,7 +360,7 @@ public ParallelIndexSubTask newSubTask(int numAttempts) getIngestionSpec(), getContext(), null, - new LocalParallelIndexTaskClientFactory(runner) + new LocalParallelIndexTaskClientFactory(supervisorTask) ); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java index 7020b4509b9a..1e8bba780cbc 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java @@ -30,7 +30,6 @@ import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; -import io.druid.indexing.common.task.TaskMonitor.TaskHistory; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.concurrent.Execs; import org.junit.After; From 77935bda7c12e92300d47048314b63fc541fe50f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 16 Jul 2018 22:44:57 -0700 Subject: [PATCH 39/47] Add more javadocs --- docs/content/ingestion/native_tasks.md | 17 ++++++- .../task/ParallelIndexSupervisorTask.java | 6 +-- .../common/task/ParallelIndexTaskRunner.java | 45 +++++++++++++++++-- .../common/task/ParallelIndexingProgress.java | 27 +++++++++++ .../SinglePhaseParallelIndexTaskRunner.java | 6 +-- ... SinglePhaseParallelIndexingProgress.java} | 34 ++++++++++++-- .../indexing/common/task/TaskHistory.java | 6 ++- .../indexing/common/task/TaskMonitor.java | 4 +- ...rallelIndexSupervisorTaskResourceTest.java | 28 ++++++------ 9 files changed, 140 insertions(+), 33 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingProgress.java rename indexing-service/src/main/java/io/druid/indexing/common/task/{ParallelIndexingStatus.java => SinglePhaseParallelIndexingProgress.java} (75%) diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 8789c50af4cd..0e9353c47038 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -156,9 +156,22 @@ The supervisor task provides some HTTP endpoints to get running status. Returns 'parallel' if the indexing task is running in parallel. Otherwise, it returns 'sequential'. -* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/status` +* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/progress` -Returns the current running status if the supervisor task is running in the parallel mode. +Returns the current progress if the supervisor task is running in the parallel mode. + +An example of the result is + +```json +{ + "running":10, + "succeeded":0, + "failed":0, + "complete":0, + "total":10, + "expectedSucceeded":10 +} +``` * `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtasks/running` diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java index d7bbde1c8750..2c049dfef1a0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java @@ -417,15 +417,15 @@ public Response getMode(@Context final HttpServletRequest req) } @GET - @Path("/status") + @Path("/progress") @Produces(MediaType.APPLICATION_JSON) - public Response getStatus(@Context final HttpServletRequest req) + public Response getProgress(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (runner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - return Response.ok(runner.getStatus()).build(); + return Response.ok(runner.getProgress()).build(); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java index 1eb07f47db8e..4d4462f174fe 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java @@ -31,30 +31,67 @@ /** * ParallelIndexTaskRunner is the actual task runner of {@link ParallelIndexSupervisorTask}. There is currently a single * implementation, i.e. {@link SinglePhaseParallelIndexTaskRunner} which supports only best-effort roll-up. We can add - * more implementations in the future. + * more implementations for different distributed indexing algorithms in the future. */ public interface ParallelIndexTaskRunner { + /** + * Runs the task. + */ TaskState run() throws Exception; + /** + * {@link PushedSegmentsReport} is the report sent by {@link ParallelIndexSubTask}s. The subTasks call this method to + * send their reports after pushing generated segments to deep storage. + */ void collectReport(PushedSegmentsReport report); - ParallelIndexingStatus getStatus(); + /** + * Returns the current {@link ParallelIndexingProgress}. + */ + ParallelIndexingProgress getProgress(); + /** + * Returns the IDs of current running tasks. + */ Set getRunningTaskIds(); + /** + * Returns all {@link SubTaskSpec}s. + */ List> getSubTaskSpecs(); + /** + * Returns running {@link SubTaskSpec}s. A {@link SubTaskSpec} is running if there is a running {@link Task} created + * using that subTaskSpec. + * + * @see SubTaskSpec#newSubTask + */ List> getRunningSubTaskSpecs(); + /** + * Returns complete {@link SubTaskSpec}s. A {@link SubTaskSpec} is complete if there is a succeeded or failed + * {@link Task} created using that subTaskSpec. + * + * @see SubTaskSpec#newSubTask + */ List> getCompleteSubTaskSpecs(); + /** + * Returns the {@link SubTaskSpec} of the given ID or null if it's not found. + */ @Nullable SubTaskSpec getSubTaskSpec(String subTaskSpecId); + /** + * Returns {@link SubTaskSpecStatus} of the given ID or null if it's not found. + */ @Nullable SubTaskSpecStatus getSubTaskState(String subTaskSpecId); + /** + * Returns {@link TaskHistory} of the given ID or null if it's not found. + */ @Nullable TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId); @@ -62,8 +99,8 @@ class SubTaskSpecStatus { private final ParallelIndexSubTaskSpec spec; @Nullable - private final TaskStatusPlus currentStatus; - private final List taskHistory; + private final TaskStatusPlus currentStatus; // null if there is no running task for the spec + private final List taskHistory; // can be empty if there is no history @JsonCreator public SubTaskSpecStatus( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingProgress.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingProgress.java new file mode 100644 index 000000000000..032e68f3074f --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingProgress.java @@ -0,0 +1,27 @@ +/* + * 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 io.druid.indexing.common.task; + +/** + * Represents the current progress of {@link ParallelIndexSupervisorTask}. Implementations can be different depending on + * the distributed indexing algorithm. + */ +interface ParallelIndexingProgress +{ +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java index cc37c6a10066..d7e7b72d4120 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java @@ -163,7 +163,7 @@ public TaskState run() throws Exception state = TaskState.SUCCESS; } else { // Failed - final ParallelIndexingStatus monitorStatus = taskMonitor.getStatus(); + final SinglePhaseParallelIndexingProgress monitorStatus = taskMonitor.getProgress(); throw new ISE( "Expected for [%d] tasks to succeed, but we got [%d] succeeded tasks and [%d] failed tasks", monitorStatus.getExpectedSucceeded(), @@ -261,9 +261,9 @@ public void collectReport(PushedSegmentsReport report) } @Override - public ParallelIndexingStatus getStatus() + public SinglePhaseParallelIndexingProgress getProgress() { - return taskMonitor == null ? ParallelIndexingStatus.notRunning() : taskMonitor.getStatus(); + return taskMonitor == null ? SinglePhaseParallelIndexingProgress.notRunning() : taskMonitor.getProgress(); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingStatus.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexingProgress.java similarity index 75% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingStatus.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexingProgress.java index 7e590e2199d7..696e8b415acd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingStatus.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexingProgress.java @@ -21,22 +21,48 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -class ParallelIndexingStatus +/** + * Represents the current progress of {@link SinglePhaseParallelIndexTaskRunner}. + */ +class SinglePhaseParallelIndexingProgress implements ParallelIndexingProgress { + /** + * Number of running tasks. + */ private final int running; + + /** + * Number of succeeded tasks. + */ private final int succeeded; + + /** + * Number of failed tasks. + */ private final int failed; + + /** + * Number of complete (succeeded + failed) tasks. + */ private final int complete; + + /** + * Number of total (running + complete) tasks. + */ private final int total; + + /** + * Number of succeeded tasks for {@link SinglePhaseParallelIndexTaskRunner} to succeed. + */ private final int expectedSucceeded; - static ParallelIndexingStatus notRunning() + static SinglePhaseParallelIndexingProgress notRunning() { - return new ParallelIndexingStatus(0, 0, 0, 0, 0, -1); + return new SinglePhaseParallelIndexingProgress(0, 0, 0, 0, 0, -1); } @JsonCreator - ParallelIndexingStatus( + SinglePhaseParallelIndexingProgress( @JsonProperty("running") int running, @JsonProperty("succeeded") int succeeded, @JsonProperty("failed") int failed, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java index b8710c27e525..fff8b04a1cc8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java @@ -25,10 +25,14 @@ import java.util.List; +/** + * Task attempt history for complete {@link SubTaskSpec}s. A {@link SubTaskSpec} is complete if its last status is + * succeeded or failed. + */ class TaskHistory { private final SubTaskSpec spec; - private final List attemptHistory; // old to recent + private final List attemptHistory; // old attempts to recent ones TaskHistory(SubTaskSpec spec, List attemptHistory) { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index b0d70223cf6c..0b8f0d455018 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -275,10 +275,10 @@ int getNumRunningTasks() } } - ParallelIndexingStatus getStatus() + SinglePhaseParallelIndexingProgress getProgress() { synchronized (taskCountLock) { - return new ParallelIndexingStatus( + return new SinglePhaseParallelIndexingProgress( numRunningTasks, numSucceededTasks, numFailedTasks, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java index dddb8bbe1faf..80f2d68ac36e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java @@ -145,13 +145,13 @@ public void testAPIs() throws Exception Assert.assertEquals("parallel", response.getEntity()); // test expectedNumSucceededTasks - response = task.getStatus(newRequest()); + response = task.getProgress(newRequest()); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(NUM_SUB_TASKS, ((ParallelIndexingStatus) response.getEntity()).getExpectedSucceeded()); + Assert.assertEquals(NUM_SUB_TASKS, ((SinglePhaseParallelIndexingProgress) response.getEntity()).getExpectedSucceeded()); // Since taskMonitor works based on polling, it's hard to use a fancier way to check its state. // We use polling to check the state of taskMonitor in this test. - while (getNumSubTasks(ParallelIndexingStatus::getRunning) < NUM_SUB_TASKS) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getRunning) < NUM_SUB_TASKS) { Thread.sleep(100); } @@ -169,7 +169,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks(ParallelIndexingStatus::getSucceeded) < succeededTasks) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -186,7 +186,7 @@ public void testAPIs() throws Exception } // Wait for new tasks to be started - while (getNumSubTasks(ParallelIndexingStatus::getFailed) < failedTasks || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) { Thread.sleep(100); } @@ -202,7 +202,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); } - while (getNumSubTasks(ParallelIndexingStatus::getSucceeded) < succeededTasks) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -221,10 +221,10 @@ public void testAPIs() throws Exception // Test one more failure runningTasks.get(0).setState(TaskState.FAILED); failedTasks++; - while (getNumSubTasks(ParallelIndexingStatus::getFailed) < failedTasks) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks) { Thread.sleep(100); } - while (getNumSubTasks(ParallelIndexingStatus::getRunning) < 1) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getRunning) < 1) { Thread.sleep(100); } @@ -237,7 +237,7 @@ public void testAPIs() throws Exception runningTasks.get(0).setState(TaskState.SUCCESS); succeededTasks++; - while (getNumSubTasks(ParallelIndexingStatus::getSucceeded) < succeededTasks) { + while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) { Thread.sleep(100); } @@ -245,11 +245,11 @@ public void testAPIs() throws Exception } @SuppressWarnings({"ConstantConditions"}) - private int getNumSubTasks(Function func) + private int getNumSubTasks(Function func) { - final Response response = task.getStatus(newRequest()); + final Response response = task.getProgress(newRequest()); Assert.assertEquals(200, response.getStatus()); - return func.apply((ParallelIndexingStatus) response.getEntity()); + return func.apply((SinglePhaseParallelIndexingProgress) response.getEntity()); } private Map buildStateMap() @@ -275,9 +275,9 @@ private void checkState( Map expectedSubTaskStateResponses // subTaskSpecId -> response ) { - Response response = task.getStatus(newRequest()); + Response response = task.getProgress(newRequest()); Assert.assertEquals(200, response.getStatus()); - final ParallelIndexingStatus monitorStatus = (ParallelIndexingStatus) response.getEntity(); + final SinglePhaseParallelIndexingProgress monitorStatus = (SinglePhaseParallelIndexingProgress) response.getEntity(); // numRunningTasks Assert.assertEquals(runningTasks.size(), monitorStatus.getRunning()); From 7e6b110dcf3e4a0d3056d8383d19a218fd6f3ff4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 17 Jul 2018 13:39:56 -0700 Subject: [PATCH 40/47] use StringUtils.nonStrictFormat for logging --- .../src/main/java/io/druid/indexing/common/IndexTaskClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java index a3aa4ab498e0..631d4b46d231 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/IndexTaskClient.java @@ -327,7 +327,7 @@ private FullResponseHolder submitRequest( } String urlForLog = (request != null ? request.getUrl().toString() - : StringUtils.format( + : StringUtils.nonStrictFormat( "%s://%s:%d%s", scheme, host, From b7266ce08d8e02ce6640379f428f2c29614ece90 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 17 Jul 2018 16:20:06 -0700 Subject: [PATCH 41/47] fix typo and remove unused class --- .../SinglePhaseParallelIndexTaskRunner.java | 4 +-- .../indexing/common/task/TaskMonitor.java | 4 +-- .../io/druid/client/indexing/ClientQuery.java | 25 ------------------- 3 files changed, 4 insertions(+), 29 deletions(-) delete mode 100644 server/src/main/java/io/druid/client/indexing/ClientQuery.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java index d7e7b72d4120..dabb39f01bec 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java @@ -311,7 +311,7 @@ public SubTaskSpec getSubTaskSpec(String subTaskSpecId) if (taskMonitor != null) { // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after // adding them to taskHistory. - final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(subTaskSpecId); + final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntry(subTaskSpecId); final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); final SubTaskSpec subTaskSpec; @@ -340,7 +340,7 @@ public SubTaskSpecStatus getSubTaskState(String subTaskSpecId) } else { // Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after // adding them to taskHistory. - final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntory(subTaskSpecId); + final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntry(subTaskSpecId); final TaskHistory taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); final SubTaskSpecStatus subTaskSpecStatus; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java index 0b8f0d455018..c519b2603b19 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java @@ -56,7 +56,7 @@ public class TaskMonitor /** * A map of subTaskSpecId to {@link MonitorEntry}. This map stores the state of running {@link SubTaskSpec}s. This is * read in {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker} and updated in {@link #submit} - * and {@link #retry}. This can also be read by calling {@link #getRunningTaskMonitorEntory}, + * and {@link #retry}. This can also be read by calling {@link #getRunningTaskMonitorEntry}, * {@link #getRunningTaskIds}, and {@link #getRunningSubTaskSpecs}. */ private final ConcurrentMap runningTasks = new ConcurrentHashMap<>(); @@ -300,7 +300,7 @@ List> getRunningSubTaskSpecs() } @Nullable - MonitorEntry getRunningTaskMonitorEntory(String subTaskSpecId) + MonitorEntry getRunningTaskMonitorEntry(String subTaskSpecId) { return runningTasks.values() .stream() diff --git a/server/src/main/java/io/druid/client/indexing/ClientQuery.java b/server/src/main/java/io/druid/client/indexing/ClientQuery.java deleted file mode 100644 index a412bc8b349d..000000000000 --- a/server/src/main/java/io/druid/client/indexing/ClientQuery.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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 io.druid.client.indexing; - -public interface ClientQuery -{ - String getId(); -} From e75baa4a6e9954fe6e8d5f95fdd46b40be15cc1a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 17 Jul 2018 16:44:18 -0700 Subject: [PATCH 42/47] fix tests --- .../task/ParallelIndexSupervisorTask.java | 15 ++++++++++- ...rallelIndexSupervisorTaskResourceTest.java | 25 +++++++------------ .../task/ParallelIndexSupervisorTaskTest.java | 12 +++++---- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java index 2c049dfef1a0..93dcf335564f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java @@ -193,6 +193,12 @@ ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) return runner; } + @VisibleForTesting + void setRunner(ParallelIndexTaskRunner runner) + { + this.runner = runner; + } + @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { @@ -221,7 +227,7 @@ static boolean isReady(TaskActionClient actionClient, SortedSet interv @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - this.toolbox = toolbox; + setToolbox(toolbox); log.info( "Found chat handler of class[%s]", @@ -245,6 +251,12 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception } } + @VisibleForTesting + void setToolbox(TaskToolbox toolbox) + { + this.toolbox = toolbox; + } + private TaskStatus runParallel(TaskToolbox toolbox) throws Exception { createRunner(toolbox); @@ -410,6 +422,7 @@ public Response getMode(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (runner == null) { + System.err.println("runner: " + runner); return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { return Response.ok(baseFirehoseFactory.isSplittable() ? "parallel" : "sequential").build(); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java index 80f2d68ac36e..404584319f61 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java @@ -100,7 +100,6 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd private ExecutorService service; private TestSupervisorTask task; - private SinglePhaseParallelIndexTaskRunner runner; @Before public void setup() throws IOException @@ -136,7 +135,7 @@ public void testAPIs() throws Exception final Future supervisorTaskFuture = service.submit(() -> task.run(toolbox)); Thread.sleep(1000); - runner = (SinglePhaseParallelIndexTaskRunner) task.getRunner(); + final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.getRunner(); Assert.assertNotNull("runner is null", runner); // test getMode @@ -479,8 +478,6 @@ public FiniteFirehoseFactory withSplit(InputSplit private class TestSupervisorTask extends TestParallelIndexSupervisorTask { - private TestRunner runner; - TestSupervisorTask( String id, TaskResource taskResource, @@ -501,22 +498,18 @@ private class TestSupervisorTask extends TestParallelIndexSupervisorTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - this.runner = new TestRunner( - toolbox, - this, - indexingServiceClient + setRunner( + new TestRunner( + toolbox, + this, + indexingServiceClient + ) ); return TaskStatus.fromCode( getId(), - runner.run() + getRunner().run() ); } - - @Override - public ParallelIndexTaskRunner getRunner() - { - return runner; - } } private class TestRunner extends TestParallelIndexTaskRunner @@ -674,7 +667,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .submit(new SurrogateAction<>(getSupervisorTaskId(), new LockListAction())); Preconditions.checkState(locks.size() == 1, "There should be a single lock"); - runner.collectReport( + task.getRunner().collectReport( new PushedSegmentsReport( getId(), Collections.singletonList( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java index 8cbfba110272..042851ec5bb3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java @@ -273,13 +273,17 @@ private static class TestSupervisorTask extends TestParallelIndexSupervisorTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - return TaskStatus.fromCode( - getId(), + setToolbox(toolbox); + setRunner( new TestRunner( toolbox, this, indexingServiceClient - ).run() + ) + ); + return TaskStatus.fromCode( + getId(), + getRunner().run() ); } } @@ -315,7 +319,6 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), supervisorTask, - this, new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( @@ -338,7 +341,6 @@ private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSp String id, String groupId, ParallelIndexSupervisorTask supervisorTask, - SinglePhaseParallelIndexTaskRunner runner, ParallelIndexIngestionSpec ingestionSpec, Map context, InputSplit inputSplit From b8ce241531ed897bd52d9b402a6ab5b603ce0cc5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 17 Jul 2018 17:07:27 -0700 Subject: [PATCH 43/47] change package --- .../io/druid/indexing/common/task/AbstractTask.java | 4 ++-- .../java/io/druid/indexing/common/task/IndexTask.java | 2 +- .../main/java/io/druid/indexing/common/task/Task.java | 2 ++ .../{ => batch/parallel}/ParallelIndexIOConfig.java | 2 +- .../parallel}/ParallelIndexIngestionSpec.java | 2 +- .../{ => batch/parallel}/ParallelIndexSubTask.java | 10 ++++++++-- .../parallel}/ParallelIndexSubTaskSpec.java | 2 +- .../parallel}/ParallelIndexSupervisorTask.java | 11 ++++++++--- .../{ => batch/parallel}/ParallelIndexTaskClient.java | 2 +- .../parallel}/ParallelIndexTaskClientFactory.java | 3 ++- .../{ => batch/parallel}/ParallelIndexTaskRunner.java | 3 ++- .../parallel}/ParallelIndexTuningConfig.java | 2 +- .../parallel}/ParallelIndexingProgress.java | 2 +- .../{ => batch/parallel}/PushedSegmentsReport.java | 2 +- .../parallel}/SinglePhaseParallelIndexTaskRunner.java | 6 +++--- .../SinglePhaseParallelIndexingProgress.java | 2 +- .../common/task/{ => batch/parallel}/SubTaskSpec.java | 3 ++- .../common/task/{ => batch/parallel}/TaskHistory.java | 3 ++- .../common/task/{ => batch/parallel}/TaskMonitor.java | 3 ++- .../AbstractParallelIndexSupervisorTaskTest.java | 7 ++++++- .../ParallelIndexSupervisorTaskResourceTest.java | 7 +++++-- .../ParallelIndexSupervisorTaskSerdeTest.java | 4 +++- .../parallel}/ParallelIndexSupervisorTaskTest.java | 3 ++- .../task/{ => batch/parallel}/TaskMonitorTest.java | 5 +++-- .../src/main/java/io/druid/cli/CliMiddleManager.java | 2 +- services/src/main/java/io/druid/cli/CliOverlord.java | 2 +- services/src/main/java/io/druid/cli/CliPeon.java | 4 ++-- 27 files changed, 65 insertions(+), 35 deletions(-) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexIOConfig.java (96%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexIngestionSpec.java (97%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexSubTask.java (97%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexSubTaskSpec.java (97%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexSupervisorTask.java (97%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexTaskClient.java (98%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexTaskClientFactory.java (94%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexTaskRunner.java (97%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexTuningConfig.java (99%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexingProgress.java (94%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/PushedSegmentsReport.java (97%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/SinglePhaseParallelIndexTaskRunner.java (98%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/SinglePhaseParallelIndexingProgress.java (98%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/SubTaskSpec.java (95%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/TaskHistory.java (95%) rename indexing-service/src/main/java/io/druid/indexing/common/task/{ => batch/parallel}/TaskMonitor.java (99%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{ => batch/parallel}/AbstractParallelIndexSupervisorTaskTest.java (97%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexSupervisorTaskResourceTest.java (98%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexSupervisorTaskSerdeTest.java (97%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{ => batch/parallel}/ParallelIndexSupervisorTaskTest.java (99%) rename indexing-service/src/test/java/io/druid/indexing/common/task/{ => batch/parallel}/TaskMonitorTest.java (97%) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java index 446078ab1f0c..3f84f6f1b805 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java @@ -77,7 +77,7 @@ protected AbstractTask( this.context = context; } - static String getOrMakeId(String id, final String typeName, String dataSource) + public static String getOrMakeId(String id, final String typeName, String dataSource) { return getOrMakeId(id, typeName, dataSource, null); } @@ -216,7 +216,7 @@ public int hashCode() return id.hashCode(); } - static List getTaskLocks(TaskActionClient client) throws IOException + public static List getTaskLocks(TaskActionClient client) throws IOException { return client.submit(new LockListAction()); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 255e2a86a1fb..acd1ca0f4d7f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -204,7 +204,7 @@ public IndexTask( ); } - IndexTask( + public IndexTask( String id, String groupId, TaskResource resource, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index f4f793473540..51d2110308b3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -24,6 +24,8 @@ import io.druid.indexer.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexSubTask; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import io.druid.query.Query; import io.druid.query.QueryRunner; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java index 18d4709c9c96..9af967201111 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIOConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java index 34fae34729c6..9912eb10a307 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexIngestionSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexIngestionSpec.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 1fca8b4df9e3..a9c7e78e1392 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; @@ -37,6 +37,12 @@ import io.druid.indexing.common.actions.SegmentAllocateAction; import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.AbstractTask; +import io.druid.indexing.common.task.ClientBasedTaskInfoProvider; +import io.druid.indexing.common.task.IndexTask; +import io.druid.indexing.common.task.IndexTaskClientFactory; +import io.druid.indexing.common.task.TaskResource; +import io.druid.indexing.common.task.Tasks; import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; @@ -79,7 +85,7 @@ */ public class ParallelIndexSubTask extends AbstractTask { - static final String TYPE = "index_sub"; + public static final String TYPE = "index_sub"; private static final Logger log = new Logger(ParallelIndexSubTask.class); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java index 0210e6f5340c..c1ee841825d9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 93dcf335564f..3c2fc722bfd9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; @@ -37,9 +37,14 @@ import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.stats.RowIngestionMetersFactory; +import io.druid.indexing.common.task.AbstractTask; +import io.druid.indexing.common.task.IndexTask; import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; -import io.druid.indexing.common.task.ParallelIndexTaskRunner.SubTaskSpecStatus; +import io.druid.indexing.common.task.IndexTaskUtils; +import io.druid.indexing.common.task.TaskResource; +import io.druid.indexing.common.task.Tasks; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; @@ -82,7 +87,7 @@ */ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHandler { - static final String TYPE = "index_parallel"; + public static final String TYPE = "index_parallel"; private static final Logger log = new Logger(ParallelIndexSupervisorTask.class); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java index 97785b166ecd..6e111607b973 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java similarity index 94% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java index 494160db40ec..645011fb1c74 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskClientFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClientFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; @@ -25,6 +25,7 @@ import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Smile; import io.druid.indexing.common.TaskInfoProvider; +import io.druid.indexing.common.task.IndexTaskClientFactory; import io.druid.java.util.http.client.HttpClient; import org.joda.time.Duration; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java index 4d4462f174fe..a8077411d61b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java @@ -17,12 +17,13 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.task.Task; import javax.annotation.Nullable; import java.util.List; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java index f060c4751c4a..b9a57e20fd13 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingProgress.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java similarity index 94% rename from indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingProgress.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java index 032e68f3074f..bb813c741d6b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ParallelIndexingProgress.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexingProgress.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; /** * Represents the current progress of {@link ParallelIndexSupervisorTask}. Implementations can be different depending on diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java index df2cb7eba3ff..548d187c5742 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/PushedSegmentsReport.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index dabb39f01bec..ae1addfe63e9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -33,8 +33,8 @@ import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import io.druid.indexing.common.task.TaskMonitor.MonitorEntry; -import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; +import io.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorEntry; +import io.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.appenderator.SegmentIdentifier; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexingProgress.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexingProgress.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java index 696e8b415acd..aa21735c2998 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SinglePhaseParallelIndexingProgress.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingProgress.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SubTaskSpec.java similarity index 95% rename from indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SubTaskSpec.java index 9d9c62b658c9..aca5164c0152 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SubTaskSpec.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/SubTaskSpec.java @@ -17,11 +17,12 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.data.input.InputSplit; +import io.druid.indexing.common.task.Task; import java.util.Map; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskHistory.java similarity index 95% rename from indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskHistory.java index fff8b04a1cc8..cb42315b1af3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskHistory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskHistory.java @@ -16,12 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.task.Task; import java.util.List; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskMonitor.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java rename to indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskMonitor.java index c519b2603b19..f0b2e3b33e02 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/TaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/TaskMonitor.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.google.common.base.Preconditions; import com.google.common.util.concurrent.Futures; @@ -27,6 +27,7 @@ import io.druid.client.indexing.TaskStatusResponse; import io.druid.indexer.TaskState; import io.druid.indexer.TaskStatusPlus; +import io.druid.indexing.common.task.Task; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java similarity index 97% rename from indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index d789440b9bdd..341d5eee6709 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -42,6 +42,11 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; +import io.druid.indexing.common.task.IndexTaskClientFactory; +import io.druid.indexing.common.task.IngestionTestBase; +import io.druid.indexing.common.task.NoopTestTaskFileWriter; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.common.task.TaskResource; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.concurrent.Execs; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java similarity index 98% rename from indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 404584319f61..0891a902dc05 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -35,7 +35,10 @@ import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SurrogateAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.common.task.ParallelIndexTaskRunner.SubTaskSpecStatus; +import io.druid.indexing.common.task.AbstractTask; +import io.druid.indexing.common.task.IndexTaskClientFactory; +import io.druid.indexing.common.task.TaskResource; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Intervals; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java similarity index 97% rename from indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 3a8f07ef843d..9ace49797e1c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; @@ -30,6 +30,8 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; +import io.druid.indexing.common.task.Task; +import io.druid.indexing.common.task.TaskResource; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.aggregation.AggregatorFactory; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java similarity index 99% rename from indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index 042851ec5bb3..b20436a158a1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import io.druid.client.indexing.IndexingServiceClient; import io.druid.data.input.FiniteFirehoseFactory; @@ -27,6 +27,7 @@ import io.druid.indexer.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.TaskResource; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java similarity index 97% rename from indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java rename to indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java index 1e8bba780cbc..33173b35feac 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.indexing.common.task; +package io.druid.indexing.common.task.batch.parallel; import com.google.common.util.concurrent.ListenableFuture; import io.druid.client.indexing.NoopIndexingServiceClient; @@ -29,7 +29,8 @@ import io.druid.indexer.TaskStatus; import io.druid.indexer.TaskStatusPlus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.task.TaskMonitor.SubTaskCompleteEvent; +import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.concurrent.Execs; import org.junit.After; diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index bf12173b231a..c802017c6391 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -46,7 +46,7 @@ import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import io.druid.indexing.common.stats.RowIngestionMetersFactory; import io.druid.indexing.common.task.IndexTaskClientFactory; -import io.druid.indexing.common.task.ParallelIndexTaskClient; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClient; import io.druid.indexing.overlord.ForkingTaskRunner; import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.worker.Worker; diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 5996ebfe13e0..724d442d6ccc 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -58,7 +58,7 @@ import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import io.druid.indexing.common.stats.RowIngestionMetersFactory; import io.druid.indexing.common.task.IndexTaskClientFactory; -import io.druid.indexing.common.task.ParallelIndexTaskClient; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClient; import io.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import io.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; import io.druid.indexing.overlord.ForkingTaskRunnerFactory; diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 791f4199a3de..1c98d08f5707 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -68,8 +68,8 @@ import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import io.druid.indexing.common.stats.RowIngestionMetersFactory; import io.druid.indexing.common.task.IndexTaskClientFactory; -import io.druid.indexing.common.task.ParallelIndexTaskClient; -import io.druid.indexing.common.task.ParallelIndexTaskClientFactory; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClient; +import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskClientFactory; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.HeapMemoryTaskStorage; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; From 3b29f6eba9531af5087651e8af5684ac57d68cef Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 18 Jul 2018 13:51:47 -0700 Subject: [PATCH 44/47] fix strict build --- .../common/task/batch/parallel/ParallelIndexSupervisorTask.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 3c2fc722bfd9..438e0f1d81ff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -427,7 +427,6 @@ public Response getMode(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (runner == null) { - System.err.println("runner: " + runner); return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { return Response.ok(baseFirehoseFactory.isSplittable() ? "parallel" : "sequential").build(); From 9853cfa398a3dce179e64a4df66eab74905832d9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 31 Jul 2018 17:18:00 -0700 Subject: [PATCH 45/47] tmp --- .../java/io/druid/indexing/overlord/TaskStorage.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java index 89849e5cf2f9..3178593c907a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java @@ -122,8 +122,8 @@ public interface TaskStorage List getActiveTasks(); /** - * Returns a list of currently running or pending tasks as stored in the storage facility as {@link TaskInfo}. No particular order - * is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. + * Returns a list of currently running or pending tasks as stored in the storage facility as {@link TaskInfo}. No + * particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation. * * @param dataSource datasource * @@ -132,9 +132,10 @@ public interface TaskStorage List> getActiveTaskInfo(@Nullable String dataSource); /** - * Returns up to {@code maxTaskStatuses} {@link TaskInfo} objects of recently finished tasks as stored in the storage facility. No - * particular order is guaranteed, but implementations are encouraged to return tasks in descending order of creation. - * No particular standard of "recent" is guaranteed, and in fact, this method is permitted to simply return nothing. + * Returns up to {@code maxTaskStatuses} {@link TaskInfo} objects of recently finished tasks as stored in the storage + * facility. No particular order is guaranteed, but implementations are encouraged to return tasks in descending order + * of creation. No particular standard of "recent" is guaranteed, and in fact, this method is permitted to simply + * return nothing. * * @param maxTaskStatuses maxTaskStatuses * @param duration duration From 20f7c7b4fb3d2d44393b1ad5188963a6d154ae99 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 1 Aug 2018 13:32:18 -0700 Subject: [PATCH 46/47] Fix overlord api according to the recent change in master --- .../MetadataStorageActionHandler.java | 3 + .../overlord/HeapMemoryTaskStorage.java | 26 +++++ .../overlord/MetadataTaskStorage.java | 7 ++ .../druid/indexing/overlord/TaskStorage.java | 3 + .../overlord/TaskStorageQueryAdapter.java | 6 ++ .../overlord/http/OverlordResource.java | 94 ++++++++----------- .../overlord/http/OverlordResourceTest.java | 22 ++--- .../SQLMetadataStorageActionHandler.java | 43 +++++++-- 8 files changed, 129 insertions(+), 75 deletions(-) diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java index 0fc43121204f..9661a641f68b 100644 --- a/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java +++ b/common/src/main/java/io/druid/metadata/MetadataStorageActionHandler.java @@ -78,6 +78,9 @@ void insert( */ Optional getStatus(String entryId); + @Nullable + TaskInfo getTaskInfo(String entryId); + /** * Return up to {@code maxNumStatuses} {@link TaskInfo} objects for all inactive entries * created on or later than the given timestamp diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java index db3a8734fdae..3493ab7de9b9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -149,6 +149,32 @@ public Optional getStatus(String taskid) } } + @Nullable + @Override + public TaskInfo getTaskInfo(String taskId) + { + giant.lock(); + + try { + Preconditions.checkNotNull(taskId, "taskId"); + final TaskStuff taskStuff = tasks.get(taskId); + if (taskStuff != null) { + return new TaskInfo<>( + taskStuff.getTask().getId(), + taskStuff.getCreatedDate(), + taskStuff.getStatus(), + taskStuff.getDataSource(), + taskStuff.getTask() + ); + } else { + return null; + } + } + finally { + giant.unlock(); + } + } + @Override public List getActiveTasks() { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java index 28f6bbcd3439..77a4c4aefa69 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/MetadataTaskStorage.java @@ -180,6 +180,13 @@ public Optional getStatus(final String taskId) return handler.getStatus(taskId); } + @Nullable + @Override + public TaskInfo getTaskInfo(String taskId) + { + return handler.getTaskInfo(taskId); + } + @Override public List getActiveTasks() { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java index 3178593c907a..521fe38dfc50 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java @@ -95,6 +95,9 @@ public interface TaskStorage */ Optional getStatus(String taskid); + @Nullable + TaskInfo getTaskInfo(String taskId); + /** * Add an action taken by a task to the audit log. * diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java index da7dd71e9360..d67d4eafa738 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -77,6 +77,12 @@ public Optional getStatus(final String taskid) return storage.getStatus(taskid); } + @Nullable + public TaskInfo getTaskInfo(String taskId) + { + return storage.getTaskInfo(taskId); + } + /** * Returns all segments created by this task. * diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 64797a1e9c03..be5fe16cc039 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -57,7 +57,6 @@ import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.EntryExistsException; @@ -102,6 +101,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; import java.util.stream.Collectors; /** @@ -249,14 +249,12 @@ public Response getTaskPayload(@PathParam("taskid") String taskid) @ResourceFilters(TaskResourceFilter.class) public Response getTaskStatus(@PathParam("taskid") String taskid) { - final Task task = taskStorageQueryAdapter.getTask(taskid).orNull(); - final TaskStatus taskStatus = taskStorageQueryAdapter.getStatus(taskid).orNull(); - final Pair createdDateAndDataSource = taskStorageQueryAdapter.getCreatedDateAndDataSource( - taskid - ); + final TaskInfo taskInfo = taskStorageQueryAdapter.getTaskInfo(taskid); TaskStatusResponse response = null; - if (task != null && taskStatus != null && createdDateAndDataSource != null) { + if (taskInfo != null) { + final BiFunction, RunnerTaskState, TaskStatusPlus> convertFn = + newTaskInfo2TaskStatusPlusFn(); if (taskMaster.getTaskRunner().isPresent()) { final TaskRunner taskRunner = taskMaster.getTaskRunner().get(); final TaskRunnerWorkItem workItem = taskRunner @@ -267,39 +265,14 @@ public Response getTaskStatus(@PathParam("taskid") String taskid) .orElse(null); if (workItem != null) { response = new TaskStatusResponse( - taskid, - new TaskStatusPlus( - taskid, - task.getType(), - createdDateAndDataSource.lhs, - workItem.getQueueInsertionTime(), - taskStatus.getStatusCode(), - taskRunner.getRunnerTaskState(taskid), - taskStatus.getDuration(), - workItem.getLocation(), - createdDateAndDataSource.rhs, - null - ) + workItem.getTaskId(), + convertFn.apply(taskInfo, taskRunner.getRunnerTaskState(workItem.getTaskId())) ); } } if (response == null) { - response = new TaskStatusResponse( - taskid, - new TaskStatusPlus( - taskid, - task.getType(), - createdDateAndDataSource.lhs, - DateTimes.EPOCH, - taskStatus.getStatusCode(), - RunnerTaskState.WAITING, - taskStatus.getDuration(), - TaskLocation.unknown(), - createdDateAndDataSource.rhs, - null - ) - ); + response = new TaskStatusResponse(taskid, convertFn.apply(taskInfo, RunnerTaskState.WAITING)); } } else { response = new TaskStatusResponse(taskid, null); @@ -658,20 +631,8 @@ public Response getTasks( null ); - Function, TaskStatusPlus> completeTaskTransformFunc = taskInfo -> new TaskStatusPlus( - taskInfo.getId(), - taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), - taskInfo.getCreatedTime(), - // Would be nice to include the real queue insertion time, but the - // TaskStorage API doesn't yet allow it. - DateTimes.EPOCH, - taskInfo.getStatus().getStatusCode(), - RunnerTaskState.NONE, - taskInfo.getStatus().getDuration(), - TaskLocation.unknown(), - taskInfo.getDataSource(), - taskInfo.getStatus().getErrorMsg() - ); + final BiFunction, RunnerTaskState, TaskStatusPlus> completeTaskTransformFunc = + newTaskInfo2TaskStatusPlusFn(); //checking for complete tasks first to avoid querying active tasks if user only wants complete tasks if (state == null || "complete".equals(StringUtils.toLowerCase(state))) { @@ -683,7 +644,10 @@ public Response getTasks( final List> taskInfoList = taskStorageQueryAdapter.getRecentlyCompletedTaskInfo( maxCompletedTasks, duration, dataSource ); - final List completedTasks = Lists.transform(taskInfoList, completeTaskTransformFunc); + final List completedTasks = new ArrayList<>(taskInfoList.size()); + taskInfoList.forEach( + taskInfo -> completedTasks.add(completeTaskTransformFunc.apply(taskInfo, RunnerTaskState.NONE)) + ); finalTaskList.addAll(completedTasks); } @@ -708,17 +672,23 @@ public Response getTasks( } if (state == null || "waiting".equals(StringUtils.toLowerCase(state))) { final List waitingWorkItems = filterActiveTasks(RunnerTaskState.WAITING, allActiveTasks); - List transformedWaitingList = Lists.transform(waitingWorkItems, activeTaskTransformFunc); + List transformedWaitingList = waitingWorkItems.stream() + .map(activeTaskTransformFunc::apply) + .collect(Collectors.toList()); finalTaskList.addAll(transformedWaitingList); } if (state == null || "pending".equals(StringUtils.toLowerCase(state))) { final List pendingWorkItems = filterActiveTasks(RunnerTaskState.PENDING, allActiveTasks); - List transformedPendingList = Lists.transform(pendingWorkItems, activeTaskTransformFunc); + List transformedPendingList = pendingWorkItems.stream() + .map(activeTaskTransformFunc::apply) + .collect(Collectors.toList()); finalTaskList.addAll(transformedPendingList); } if (state == null || "running".equals(StringUtils.toLowerCase(state))) { final List runningWorkItems = filterActiveTasks(RunnerTaskState.RUNNING, allActiveTasks); - List transformedRunningList = Lists.transform(runningWorkItems, activeTaskTransformFunc); + List transformedRunningList = runningWorkItems.stream() + .map(activeTaskTransformFunc::apply) + .collect(Collectors.toList()); finalTaskList.addAll(transformedRunningList); } final List authorizedList = securedTaskStatusPlus( @@ -730,6 +700,24 @@ public Response getTasks( return Response.ok(authorizedList).build(); } + private static BiFunction, RunnerTaskState, TaskStatusPlus> newTaskInfo2TaskStatusPlusFn() + { + return (taskInfo, runnerTaskState) -> new TaskStatusPlus( + taskInfo.getId(), + taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), + taskInfo.getCreatedTime(), + // Would be nice to include the real queue insertion time, but the + // TaskStorage API doesn't yet allow it. + DateTimes.EPOCH, + taskInfo.getStatus().getStatusCode(), + runnerTaskState, + taskInfo.getStatus().getDuration(), + TaskLocation.unknown(), + taskInfo.getDataSource(), + taskInfo.getStatus().getErrorMsg() + ); + } + private List filterActiveTasks( RunnerTaskState state, List allTasks diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index cbedaf61255e..40588cea9b4e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -40,7 +40,6 @@ import io.druid.indexing.overlord.TaskRunnerWorkItem; import io.druid.indexing.overlord.TaskStorageQueryAdapter; import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.Pair; import io.druid.segment.TestHelper; import io.druid.server.security.Access; import io.druid.server.security.Action; @@ -863,22 +862,13 @@ public void testGetTaskPayload() throws Exception public void testGetTaskStatus() throws Exception { expectAuthorizationTokenCheck(); - EasyMock.expect(taskStorageQueryAdapter.getTask("mytask")) - .andReturn(Optional.of(NoopTask.create("mytask", 0))); - - EasyMock.expect(taskStorageQueryAdapter.getStatus("mytask")) - .andReturn(Optional.of(TaskStatus.running("mytask"))); + final Task task = NoopTask.create("mytask", 0); + final TaskStatus status = TaskStatus.running("mytask"); - EasyMock.expect(taskStorageQueryAdapter.getCreatedDateAndDataSource("mytask")) - .andReturn(Pair.of(DateTimes.of("2018-01-01"), "mydatasource")); - - EasyMock.expect(taskStorageQueryAdapter.getTask("othertask")) - .andReturn(Optional.absent()); - - EasyMock.expect(taskStorageQueryAdapter.getStatus("othertask")) - .andReturn(Optional.absent()); + EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("mytask")) + .andReturn(new TaskInfo<>(task.getId(), DateTimes.of("2018-01-01"), status, task.getDataSource(), task)); - EasyMock.expect(taskStorageQueryAdapter.getCreatedDateAndDataSource("othertask")) + EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("othertask")) .andReturn(null); EasyMock.>expect(taskRunner.getKnownTasks()) @@ -903,7 +893,7 @@ public void testGetTaskStatus() throws Exception RunnerTaskState.RUNNING, -1L, TaskLocation.unknown(), - "mydatasource", + task.getDataSource(), null ) ), diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java index f940fe3c2217..4717367e0fd6 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataStorageActionHandler.java @@ -68,6 +68,8 @@ public abstract class SQLMetadataStorageActionHandler taskInfoMapper; + public SQLMetadataStorageActionHandler( final SQLMetadataConnector connector, final ObjectMapper jsonMapper, @@ -88,6 +90,7 @@ public SQLMetadataStorageActionHandler( this.entryTable = entryTable; this.logTable = logTable; this.lockTable = lockTable; + this.taskInfoMapper = new TaskInfoMapper<>(jsonMapper, entryType, statusType); } protected SQLMetadataConnector getConnector() @@ -235,6 +238,22 @@ public Optional withHandle(Handle handle) throws Exception ); } + @Override + @Nullable + public TaskInfo getTaskInfo(String entryId) + { + return connector.retryWithHandle(handle -> { + final String query = StringUtils.format( + "SELECT id, status_payload, payload, datasource, created_date FROM %s WHERE id = :id", + entryTable + ); + return handle.createQuery(query) + .bind("id", entryId) + .map(taskInfoMapper) + .first(); + }); + } + @Override public List> getCompletedTaskInfo( DateTime timestamp, @@ -250,7 +269,7 @@ public List> getCompletedTaskInfo( maxNumStatuses, dataSource ); - return query.map(new TaskInfoMapper()).list(); + return query.map(taskInfoMapper).list(); } ); } @@ -264,7 +283,7 @@ public List> getActiveTaskInfo(@Nullable String handle, dataSource ); - return query.map(new TaskInfoMapper()).list(); + return query.map(taskInfoMapper).list(); } ); } @@ -302,23 +321,35 @@ private String getWhereClauseForActiveStatusesQuery(String dataSource) return sql; } - class TaskInfoMapper implements ResultSetMapper> + static class TaskInfoMapper implements ResultSetMapper> { + private final ObjectMapper objectMapper; + private final TypeReference entryType; + private final TypeReference statusType; + + TaskInfoMapper(ObjectMapper objectMapper, TypeReference entryType, TypeReference statusType) + { + this.objectMapper = objectMapper; + this.entryType = entryType; + this.statusType = statusType; + } + @Override - public TaskInfo map(int index, ResultSet resultSet, StatementContext context) throws SQLException + public TaskInfo map(int index, ResultSet resultSet, StatementContext context) + throws SQLException { final TaskInfo taskInfo; EntryType task; StatusType status; try { - task = getJsonMapper().readValue(resultSet.getBytes("payload"), getEntryType()); + task = objectMapper.readValue(resultSet.getBytes("payload"), entryType); } catch (IOException e) { log.error(e, "Encountered exception while deserializing task payload, setting task to null"); task = null; } try { - status = getJsonMapper().readValue(resultSet.getBytes("status_payload"), getStatusType()); + status = objectMapper.readValue(resultSet.getBytes("status_payload"), statusType); } catch (IOException e) { log.error(e, "Encountered exception while deserializing task status_payload"); From 826ae5448d381e8cc79228ea307c001c917e515b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 1 Aug 2018 14:24:23 -0700 Subject: [PATCH 47/47] Fix it test --- .../overlord/http/OverlordResource.java | 58 +++++++++++++++---- 1 file changed, 48 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index be5fe16cc039..7332ec59d7f5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -253,8 +253,6 @@ public Response getTaskStatus(@PathParam("taskid") String taskid) TaskStatusResponse response = null; if (taskInfo != null) { - final BiFunction, RunnerTaskState, TaskStatusPlus> convertFn = - newTaskInfo2TaskStatusPlusFn(); if (taskMaster.getTaskRunner().isPresent()) { final TaskRunner taskRunner = taskMaster.getTaskRunner().get(); final TaskRunnerWorkItem workItem = taskRunner @@ -266,13 +264,42 @@ public Response getTaskStatus(@PathParam("taskid") String taskid) if (workItem != null) { response = new TaskStatusResponse( workItem.getTaskId(), - convertFn.apply(taskInfo, taskRunner.getRunnerTaskState(workItem.getTaskId())) + new TaskStatusPlus( + taskInfo.getId(), + taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), + taskInfo.getCreatedTime(), + // Would be nice to include the real queue insertion time, but the + // TaskStorage API doesn't yet allow it. + DateTimes.EPOCH, + taskInfo.getStatus().getStatusCode(), + taskRunner.getRunnerTaskState(workItem.getTaskId()), + taskInfo.getStatus().getDuration(), + workItem.getLocation(), + taskInfo.getDataSource(), + taskInfo.getStatus().getErrorMsg() + ) ); } } if (response == null) { - response = new TaskStatusResponse(taskid, convertFn.apply(taskInfo, RunnerTaskState.WAITING)); + response = new TaskStatusResponse( + taskid, + new TaskStatusPlus( + taskInfo.getId(), + taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), + taskInfo.getCreatedTime(), + // Would be nice to include the real queue insertion time, but the + // TaskStorage API doesn't yet allow it. + DateTimes.EPOCH, + taskInfo.getStatus().getStatusCode(), + RunnerTaskState.WAITING, + taskInfo.getStatus().getDuration(), + TaskLocation.unknown(), + taskInfo.getDataSource(), + taskInfo.getStatus().getErrorMsg() + ) + ); } } else { response = new TaskStatusResponse(taskid, null); @@ -631,8 +658,20 @@ public Response getTasks( null ); - final BiFunction, RunnerTaskState, TaskStatusPlus> completeTaskTransformFunc = - newTaskInfo2TaskStatusPlusFn(); + Function, TaskStatusPlus> completeTaskTransformFunc = taskInfo -> new TaskStatusPlus( + taskInfo.getId(), + taskInfo.getTask() == null ? null : taskInfo.getTask().getType(), + taskInfo.getCreatedTime(), + // Would be nice to include the real queue insertion time, but the + // TaskStorage API doesn't yet allow it. + DateTimes.EPOCH, + taskInfo.getStatus().getStatusCode(), + RunnerTaskState.NONE, + taskInfo.getStatus().getDuration(), + TaskLocation.unknown(), + taskInfo.getDataSource(), + taskInfo.getStatus().getErrorMsg() + ); //checking for complete tasks first to avoid querying active tasks if user only wants complete tasks if (state == null || "complete".equals(StringUtils.toLowerCase(state))) { @@ -644,10 +683,9 @@ public Response getTasks( final List> taskInfoList = taskStorageQueryAdapter.getRecentlyCompletedTaskInfo( maxCompletedTasks, duration, dataSource ); - final List completedTasks = new ArrayList<>(taskInfoList.size()); - taskInfoList.forEach( - taskInfo -> completedTasks.add(completeTaskTransformFunc.apply(taskInfo, RunnerTaskState.NONE)) - ); + final List completedTasks = taskInfoList.stream() + .map(completeTaskTransformFunc::apply) + .collect(Collectors.toList()); finalTaskList.addAll(completedTasks); }