From 8e6040a19b9b0149577ebfee86bc626aedfd9db5 Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Sun, 10 Nov 2019 20:35:05 -0800 Subject: [PATCH 01/12] Refactor parallel indexing perfect rollup partitioning Refactoring to make it easier to later add range partitioning for perfect rollup parallel indexing. This is accomplished by adding several new base classes (e.g., PerfectRollupWorkerTask) and new classes for encapsulating logic that needs to be changed for different partitioning strategies (e.g., IndexTaskInputRowIteratorBuilder). The code is functionally equivalent to before except for the following small behavior changes: 1) PartialSegmentMergeTask: Previously, this task had a priority of DEFAULT_TASK_PRIORITY. It now has a priority of DEFAULT_BATCH_INDEX_TASK_PRIORITY (via the new PerfectRollupWorkerTask base class), since it is a batch index task. 2) ParallelIndexPhaseRunner: A decorator was added to subTaskSpecIterator to ensure the subtasks are generated with unique ids. Previously, only tests (i.e., MultiPhaseParallelIndexingTest) would have this decorator, but this behavior is desired for non-test code as well. --- .../druid/data/input/InputRowIterator.java | 100 +++ .../partitions/DynamicPartitionsSpec.java | 6 + .../partitions/HashedPartitionsSpec.java | 8 + .../indexer/partitions/PartitionsSpec.java | 32 + .../SingleDimensionPartitionsSpec.java | 12 +- .../data/input/InputRowIteratorTest.java | 182 ++++++ .../testing/junit/LoggerCaptureRule.java | 11 +- indexing-service/pom.xml | 2 +- .../task/CachingLocalSegmentAllocator.java | 103 +--- .../DefaultCachingLocalSegmentAllocator.java | 114 ++++ .../common/task/FiniteFirehoseProcessor.java | 39 +- .../druid/indexing/common/task/IndexTask.java | 30 +- .../druid/indexing/common/task/Task.java | 8 +- ...FirehoseSplitParallelIndexTaskRunner.java} | 74 +-- .../parallel/GeneratedPartitionsReport.java | 8 +- .../batch/parallel/HashPartitionLocation.java | 54 ++ .../batch/parallel/HashPartitionStat.java | 90 +++ .../parallel/ParallelIndexPhaseRunner.java | 43 +- .../parallel/ParallelIndexSupervisorTask.java | 125 ++-- .../parallel/ParallelIndexTaskRunner.java | 2 +- .../parallel/ParallelIndexTuningConfig.java | 4 +- ...egmentGenerateParallelIndexTaskRunner.java | 115 ++++ .../PartialHashSegmentGenerateTask.java | 165 +++++ ...hSegmentMergeParallelIndexTaskRunner.java} | 37 +- .../parallel/PartialHashSegmentMergeTask.java | 114 ++++ .../parallel/PartialSegmentGenerateTask.java | 195 ++---- .../parallel/PartialSegmentMergeIOConfig.java | 10 +- .../PartialSegmentMergeIngestionSpec.java | 6 +- .../parallel/PartialSegmentMergeTask.java | 172 ++---- .../batch/parallel/PartitionLocation.java | 59 +- .../task/batch/parallel/PartitionStat.java | 53 +- .../parallel/PerfectRollupWorkerTask.java | 129 ++++ ...faultIndexTaskInputRowIteratorBuilder.java | 148 +++++ .../IndexTaskInputRowIteratorBuilder.java | 60 ++ .../common/task/IndexTaskSerdeTest.java | 2 +- ...bstractMultiPhaseParallelIndexingTest.java | 263 ++++++++ ...stractParallelIndexSupervisorTaskTest.java | 29 +- .../common/task/batch/parallel/Factory.java | 273 ++++++++ .../GeneratedPartitionsReportTest.java | 66 ++ ...rtitionMultiPhaseParallelIndexingTest.java | 334 ++++++++++ .../batch/parallel/HashPartitionStatTest.java | 59 ++ .../MultiPhaseParallelIndexingTest.java | 583 ------------------ .../ParallelIndexSupervisorTaskSerdeTest.java | 48 +- .../PartialHashSegmentGenerateTaskTest.java | 70 +++ .../PartialHashSegmentMergeTaskTest.java | 90 +++ .../parallel/PerfectRollupWorkerTaskTest.java | 189 ++++++ ...tIndexTaskInputRowIteratorBuilderTest.java | 216 +++++++ .../task/batch/parallel/iterator/Factory.java | 151 +++++ .../indexer/AbstractITBatchIndexTest.java | 8 +- .../tests/indexer/ITParallelIndexTest.java | 12 +- .../org/apache/druid/jackson/JodaStuff.java | 10 + .../org/apache/druid/segment/TestHelper.java | 61 ++ .../firehose/InlineFirehoseFactory.java | 4 +- 53 files changed, 3574 insertions(+), 1204 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/data/input/InputRowIterator.java create mode 100644 core/src/test/java/org/apache/druid/data/input/InputRowIteratorTest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/DefaultCachingLocalSegmentAllocator.java rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/{PartialSegmentGenerateParallelIndexTaskRunner.java => FirehoseSplitParallelIndexTaskRunner.java} (63%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/{PartialSegmentMergeParallelIndexTaskRunner.java => PartialHashSegmentMergeParallelIndexTaskRunner.java} (67%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/Factory.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReportTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowIterator.java b/core/src/main/java/org/apache/druid/data/input/InputRowIterator.java new file mode 100644 index 000000000000..a6c526b4b478 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/InputRowIterator.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * {@link Iterator} for {@link InputRow}s from a {@link Firehose}. + */ +public class InputRowIterator implements Iterator +{ + @FunctionalInterface + public interface InputRowHandler + { + /** + * @return True if inputRow was successfully handled and no further processing is needed + */ + boolean handle(InputRow inputRow); + } + + private final Firehose firehose; + private final List inputRowHandlers; + + /** + * @param firehose Source of {@link InputRow}s + * @param inputRowHandlers Before yielding the next {@link InputRow}, each {@link InputRowHandler} is sequentially + * applied to the {@link InputRow} until one of them returns true or all of the handlers are + * applied. + */ + public InputRowIterator( + Firehose firehose, + List inputRowHandlers + ) + { + this.firehose = firehose; + this.inputRowHandlers = inputRowHandlers; + } + + @Override + public boolean hasNext() + { + try { + return firehose.hasMore(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + /** + * @return Next {@link InputRow} or null if row was successfully handled by an {@link InputRowHandler}. + */ + @Override + @Nullable + public InputRow next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + InputRow inputRow; + try { + inputRow = firehose.nextRow(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + + for (InputRowHandler inputRowHandler : inputRowHandlers) { + if (inputRowHandler.handle(inputRow)) { + return null; + } + } + + return inputRow; + } +} + diff --git a/core/src/main/java/org/apache/druid/indexer/partitions/DynamicPartitionsSpec.java b/core/src/main/java/org/apache/druid/indexer/partitions/DynamicPartitionsSpec.java index 141af59d1c67..5ec443aa4cdf 100644 --- a/core/src/main/java/org/apache/druid/indexer/partitions/DynamicPartitionsSpec.java +++ b/core/src/main/java/org/apache/druid/indexer/partitions/DynamicPartitionsSpec.java @@ -81,6 +81,12 @@ public boolean needsDeterminePartitions(boolean useForHadoopTask) return false; } + @Override + public String getForceGuaranteedRollupIncompatiblityReason() + { + return NAME + " partitions unsupported"; + } + @Override public boolean equals(Object o) { diff --git a/core/src/main/java/org/apache/druid/indexer/partitions/HashedPartitionsSpec.java b/core/src/main/java/org/apache/druid/indexer/partitions/HashedPartitionsSpec.java index bad9cd95ce6d..26c3365e6425 100644 --- a/core/src/main/java/org/apache/druid/indexer/partitions/HashedPartitionsSpec.java +++ b/core/src/main/java/org/apache/druid/indexer/partitions/HashedPartitionsSpec.java @@ -37,6 +37,8 @@ public class HashedPartitionsSpec implements DimensionBasedPartitionsSpec @VisibleForTesting static final String NUM_SHARDS = "numShards"; + private static final String FORCE_GUARANTEED_ROLLUP_COMPATIBLE = ""; + @Nullable private final Integer maxRowsPerSegment; @Nullable @@ -149,6 +151,12 @@ public List getPartitionDimensions() return partitionDimensions; } + @Override + public String getForceGuaranteedRollupIncompatiblityReason() + { + return getNumShards() == null ? NUM_SHARDS + " must be specified" : FORCE_GUARANTEED_ROLLUP_COMPATIBLE; + } + @Override public boolean equals(Object o) { diff --git a/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java b/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java index 11f9ec81212a..3c82ed49349a 100644 --- a/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java +++ b/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java @@ -19,6 +19,7 @@ package org.apache.druid.indexer.partitions; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; @@ -75,4 +76,35 @@ static Integer resolveHistoricalNullIfNeeded(@Nullable Integer val) { return isEffectivelyNull(val) ? null : val; } + + /** + * @return True if this partitionSpec's type is compatible with forceGuaranteedRollup=true. + * + * @see org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig#forceGuaranteedRollup + */ + @JsonIgnore + default boolean isForceGuaranteedRollupCompatibleType() + { + return !(this instanceof DynamicPartitionsSpec); + } + + /** + * @return True if this partitionSpec's property values are compatible with forceGuaranteedRollup=true. + * + * @see org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig#forceGuaranteedRollup + */ + @JsonIgnore + default boolean isForceGuaranteedRollupCompatible() + { + return getForceGuaranteedRollupIncompatiblityReason().isEmpty(); + } + + /** + * @return Message describing why this partitionSpec is incompatible with forceGuaranteedRollup=true. Empty string if + * the partitionSpec is compatible. + * + * @see org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig#forceGuaranteedRollup + */ + @JsonIgnore + String getForceGuaranteedRollupIncompatiblityReason(); } diff --git a/core/src/main/java/org/apache/druid/indexer/partitions/SingleDimensionPartitionsSpec.java b/core/src/main/java/org/apache/druid/indexer/partitions/SingleDimensionPartitionsSpec.java index f4fe698c55b7..d193b30b2cf1 100644 --- a/core/src/main/java/org/apache/druid/indexer/partitions/SingleDimensionPartitionsSpec.java +++ b/core/src/main/java/org/apache/druid/indexer/partitions/SingleDimensionPartitionsSpec.java @@ -39,10 +39,12 @@ */ public class SingleDimensionPartitionsSpec implements DimensionBasedPartitionsSpec { - static final String NAME = "single_dim"; + public static final String NAME = "single_dim"; static final String OLD_NAME = "dimension"; // for backward compatibility + private static final String PARITION_DIMENSION = "partitionDimension"; private static final String MAX_PARTITION_SIZE = "maxPartitionSize"; + private static final String FORCE_GUARANTEED_ROLLUP_COMPATIBLE = ""; private final Integer targetRowsPerSegment; private final Integer maxRowsPerSegment; @@ -56,7 +58,7 @@ public class SingleDimensionPartitionsSpec implements DimensionBasedPartitionsSp public SingleDimensionPartitionsSpec( @JsonProperty(DimensionBasedPartitionsSpec.TARGET_ROWS_PER_SEGMENT) @Nullable Integer targetRowsPerSegment, @JsonProperty(PartitionsSpec.MAX_ROWS_PER_SEGMENT) @Nullable Integer maxRowsPerSegment, - @JsonProperty("partitionDimension") @Nullable String partitionDimension, + @JsonProperty(PARITION_DIMENSION) @Nullable String partitionDimension, @JsonProperty("assumeGrouped") boolean assumeGrouped, // false by default // Deprecated properties preserved for backward compatibility: @@ -170,6 +172,12 @@ public List getPartitionDimensions() return partitionDimension == null ? Collections.emptyList() : Collections.singletonList(partitionDimension); } + @Override + public String getForceGuaranteedRollupIncompatiblityReason() + { + return NAME + " partitions unsupported"; + } + @Override public boolean needsDeterminePartitions(boolean useForHadoopTask) { diff --git a/core/src/test/java/org/apache/druid/data/input/InputRowIteratorTest.java b/core/src/test/java/org/apache/druid/data/input/InputRowIteratorTest.java new file mode 100644 index 000000000000..52cd8fffe435 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/InputRowIteratorTest.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input; + +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.runner.RunWith; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +@RunWith(Enclosed.class) +public class InputRowIteratorTest +{ + public static class AbsentRowTest + { + private static final Firehose EMPTY_FIREHOSE = new TestFirehose() + { + @Override + public boolean hasMore() + { + return false; + } + + @Nullable + @Override + public InputRow nextRow() + { + return null; + } + }; + + private InputRowIterator target; + + @Before + public void setup() + { + target = new InputRowIterator(EMPTY_FIREHOSE, Collections.emptyList()); + } + + @Test + public void doesNotHaveNext() + { + Assert.assertFalse(target.hasNext()); + } + + @Test(expected = NoSuchElementException.class) + public void throwsExceptionWhenYieldingNext() + { + target.next(); + } + } + + public static class PresentRowTest + { + private static final InputRow INPUT_ROW1 = EasyMock.mock(InputRow.class); + private static final InputRow INPUT_ROW2 = EasyMock.mock(InputRow.class); + private static final List INPUT_ROWS = Arrays.asList(INPUT_ROW1, INPUT_ROW2); + + private TestInputRowHandler successfulHandler; + private TestInputRowHandler unsuccessfulHandler; + + @Before + public void setup() + { + successfulHandler = new TestInputRowHandler(true); + unsuccessfulHandler = new TestInputRowHandler(false); + } + + @Test + public void hasNext() + { + InputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); + Assert.assertTrue(target.hasNext()); + Assert.assertFalse(unsuccessfulHandler.invoked); + } + + @Test + public void yieldsNextIfUnhandled() + { + InputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); + Assert.assertEquals(INPUT_ROW1, target.next()); + Assert.assertTrue(unsuccessfulHandler.invoked); + } + + @Test + public void yieldsNullIfHandledByFirst() + { + InputRowIterator target = createInputRowIterator(successfulHandler, unsuccessfulHandler); + Assert.assertNull(target.next()); + Assert.assertTrue(successfulHandler.invoked); + Assert.assertFalse(unsuccessfulHandler.invoked); + } + + @Test + public void yieldsNullIfHandledBySecond() + { + InputRowIterator target = createInputRowIterator(unsuccessfulHandler, successfulHandler); + Assert.assertNull(target.next()); + Assert.assertTrue(unsuccessfulHandler.invoked); + Assert.assertTrue(successfulHandler.invoked); + } + + private static InputRowIterator createInputRowIterator( + InputRowIterator.InputRowHandler firstHandler, + InputRowIterator.InputRowHandler secondHandler + ) + { + Firehose firehose = new TestFirehose() + { + private final Iterator delegate = INPUT_ROWS.iterator(); + + @Override + public boolean hasMore() + { + return delegate.hasNext(); + } + + @Nullable + @Override + public InputRow nextRow() + { + return delegate.next(); + } + }; + + return new InputRowIterator(firehose, Arrays.asList(firstHandler, secondHandler)); + } + + private static class TestInputRowHandler implements InputRowIterator.InputRowHandler + { + boolean invoked = false; + + private final boolean successful; + + TestInputRowHandler(boolean successful) + { + this.successful = successful; + } + + @Override + public boolean handle(InputRow inputRow) + { + invoked = true; + return successful; + } + } + } + + private abstract static class TestFirehose implements Firehose + { + @Override + public void close() + { + throw new UnsupportedOperationException(); + } + } +} diff --git a/core/src/test/java/org/apache/druid/testing/junit/LoggerCaptureRule.java b/core/src/test/java/org/apache/druid/testing/junit/LoggerCaptureRule.java index 4e533650e992..ae7a1dfde055 100644 --- a/core/src/test/java/org/apache/druid/testing/junit/LoggerCaptureRule.java +++ b/core/src/test/java/org/apache/druid/testing/junit/LoggerCaptureRule.java @@ -49,7 +49,7 @@ public LoggerCaptureRule(Class targetClass) @Override protected void before() { - inMemoryAppender = new InMemoryAppender(); + inMemoryAppender = new InMemoryAppender(targetClass); LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false); Configuration configuration = loggerContext.getConfiguration(); targetClassLoggerConfig = configuration.getLoggerConfig(targetClass.getName()); @@ -77,19 +77,24 @@ private static class InMemoryAppender extends AbstractAppender { static final String NAME = InMemoryAppender.class.getName(); + private final String targetLoggerName; + // logEvents has concurrent iteration and modification in CuratorModuleTest::exitsJvmWhenMaxRetriesExceeded(), needs to be thread safe private final CopyOnWriteArrayList logEvents; - InMemoryAppender() + InMemoryAppender(Class targetClass) { super(NAME, null, null); + targetLoggerName = targetClass.getName(); logEvents = new CopyOnWriteArrayList<>(); } @Override public void append(LogEvent logEvent) { - logEvents.add(logEvent); + if (logEvent.getLoggerName().equals(targetLoggerName)) { + logEvents.add(logEvent); + } } List getLogEvents() diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 11be3ace6035..d7df2c07fad5 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -239,7 +239,7 @@ org.hamcrest - hamcrest-core + hamcrest-all test diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index 80082ddea21b..c9a87bc26cc5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java @@ -19,62 +19,70 @@ package org.apache.druid.indexing.common.task; -import com.google.common.base.Preconditions; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.ShardSpec; -import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.Interval; -import javax.annotation.Nullable; 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.function.Function; import java.util.stream.Collectors; -import java.util.stream.IntStream; /** - * Allocates all necessary segments locally at the beginning and reuse them. + * Allocates all necessary segments locally at the beginning and reuses them. + * + * @see DefaultCachingLocalSegmentAllocator + * @see RangePartitionCachingLocalSegmentAllocator */ -public class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator +class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator { - private final TaskToolbox toolbox; private final String taskId; - private final String dataSource; - private final Map> allocateSpec; - @Nullable + private final Map sequenceNameToSegmentId; private final ShardSpecs shardSpecs; - // sequenceName -> segmentId - private final Map sequenceNameToSegmentId; + @FunctionalInterface + interface IntervalToSegmentIdsCreator + { + /** + * @param versionFinder Returns the version for the specified interval + * @return Information for segment preallocation + */ + Map> create(Function versionFinder); + } - public CachingLocalSegmentAllocator( + CachingLocalSegmentAllocator( TaskToolbox toolbox, String taskId, - String dataSource, - Map> allocateSpec + IntervalToSegmentIdsCreator intervalToSegmentIdsCreator ) throws IOException { - this.toolbox = toolbox; this.taskId = taskId; - this.dataSource = dataSource; - this.allocateSpec = allocateSpec; this.sequenceNameToSegmentId = new HashMap<>(); - final Map> intervalToIds = getIntervalToSegmentIds(); + final Map intervalToVersion = toolbox.getTaskActionClient() + .submit(new LockListAction()) + .stream() + .collect(Collectors.toMap( + TaskLock::getInterval, + TaskLock::getVersion + )); + Function versionFinder = interval -> findVersion(intervalToVersion, interval); + + final Map> intervalToIds = intervalToSegmentIdsCreator.create(versionFinder); final Map> shardSpecMap = new HashMap<>(); - for (Map.Entry> entry : intervalToIds.entrySet()) { + for (Entry> entry : intervalToIds.entrySet()) { final Interval interval = entry.getKey(); final List idsPerInterval = intervalToIds.get(interval); @@ -87,38 +95,6 @@ public CachingLocalSegmentAllocator( shardSpecs = new ShardSpecs(shardSpecMap); } - private Map> getIntervalToSegmentIds() throws IOException - { - final Map intervalToVersion = getToolbox().getTaskActionClient() - .submit(new LockListAction()) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - final Map> allocateSpec = getAllocateSpec(); - final Map> intervalToSegmentIds = new HashMap<>(allocateSpec.size()); - for (Entry> entry : allocateSpec.entrySet()) { - final Interval interval = entry.getKey(); - final ShardSpecFactory shardSpecFactory = entry.getValue().lhs; - final int numSegmentsToAllocate = Preconditions.checkNotNull( - entry.getValue().rhs, - "numSegmentsToAllocate for interval[%s]", - interval - ); - - intervalToSegmentIds.put( - interval, - IntStream.range(0, numSegmentsToAllocate) - .mapToObj(i -> new SegmentIdWithShardSpec( - getDataSource(), - interval, - findVersion(intervalToVersion, interval), - shardSpecFactory.create(getToolbox().getObjectMapper(), i) - )) - .collect(Collectors.toList()) - ); - } - return intervalToSegmentIds; - } - private static String findVersion(Map intervalToVersion, Interval interval) { return intervalToVersion.entrySet().stream() @@ -128,27 +104,6 @@ private static String findVersion(Map intervalToVersion, Inter .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); } - - TaskToolbox getToolbox() - { - return toolbox; - } - - String getTaskId() - { - return taskId; - } - - String getDataSource() - { - return dataSource; - } - - Map> getAllocateSpec() - { - return allocateSpec; - } - @Override public SegmentIdWithShardSpec allocate( InputRow row, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/DefaultCachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/DefaultCachingLocalSegmentAllocator.java new file mode 100644 index 000000000000..952d099a1b00 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/DefaultCachingLocalSegmentAllocator.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Allocates all necessary segments locally at the beginning and reuses them. + * + * @see CachingLocalSegmentAllocator + */ +public class DefaultCachingLocalSegmentAllocator implements IndexTaskSegmentAllocator +{ + private final TaskToolbox toolbox; + private final String dataSource; + private final Map> allocateSpec; + private final IndexTaskSegmentAllocator delegate; + + public DefaultCachingLocalSegmentAllocator( + TaskToolbox toolbox, + String taskId, + String dataSource, + Map> allocateSpec + ) throws IOException + { + this.toolbox = toolbox; + this.dataSource = dataSource; + this.allocateSpec = allocateSpec; + + this.delegate = new CachingLocalSegmentAllocator( + toolbox, + taskId, + this::getIntervalToSegmentIds + ); + } + + private Map> getIntervalToSegmentIds(Function versionFinder) + { + final Map> intervalToSegmentIds = + Maps.newHashMapWithExpectedSize(allocateSpec.size()); + + for (Entry> entry : allocateSpec.entrySet()) { + final Interval interval = entry.getKey(); + final ShardSpecFactory shardSpecFactory = entry.getValue().lhs; + final int numSegmentsToAllocate = Preconditions.checkNotNull( + entry.getValue().rhs, + "numSegmentsToAllocate for interval[%s]", + interval + ); + + intervalToSegmentIds.put( + interval, + IntStream.range(0, numSegmentsToAllocate) + .mapToObj(i -> new SegmentIdWithShardSpec( + dataSource, + interval, + versionFinder.apply(interval), + shardSpecFactory.create(toolbox.getObjectMapper(), i) + )) + .collect(Collectors.toList()) + ); + } + return intervalToSegmentIds; + } + + @Override + public String getSequenceName(Interval interval, InputRow inputRow) + { + return delegate.getSequenceName(interval, inputRow); + } + + @Override + public SegmentIdWithShardSpec allocate( + InputRow row, + String sequenceName, + String previousSegmentId, + boolean skipSegmentLineageCheck + ) throws IOException + { + return delegate.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java index a3463f41ecc0..dde212852951 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java @@ -23,12 +23,12 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowIterator; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.indexing.DataSchema; @@ -55,13 +55,15 @@ public class FiniteFirehoseProcessor private final boolean logParseExceptions; private final int maxParseExceptions; private final long pushTimeout; + private final IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder; public FiniteFirehoseProcessor( RowIngestionMeters buildSegmentsMeters, @Nullable CircularBuffer buildSegmentsSavedParseExceptions, boolean logParseExceptions, int maxParseExceptions, - long pushTimeout + long pushTimeout, + IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder ) { this.buildSegmentsMeters = buildSegmentsMeters; @@ -69,6 +71,7 @@ public FiniteFirehoseProcessor( this.logParseExceptions = logParseExceptions; this.maxParseExceptions = maxParseExceptions; this.pushTimeout = pushTimeout; + this.inputRowIteratorBuilder = inputRowIteratorBuilder; } /** @@ -92,32 +95,26 @@ public SegmentsAndMetadata process( ? (DynamicPartitionsSpec) partitionsSpec : null; final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + try ( final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { - while (firehose.hasMore()) { + InputRowIterator iterator = inputRowIteratorBuilder + .firehose(firehose) + .granularitySpec(granularitySpec) + .nullRowRunnable(buildSegmentsMeters::incrementThrownAway) + .absentBucketIntervalConsumer(inputRow -> buildSegmentsMeters.incrementThrownAway()) + .build(); + + while (iterator.hasNext()) { try { - final InputRow inputRow = firehose.nextRow(); - + InputRow inputRow = iterator.next(); if (inputRow == null) { - buildSegmentsMeters.incrementThrownAway(); - continue; - } - - if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) { - final String errorMsg = StringUtils.format( - "Encountered row with timestamp that cannot be represented as a long: [%s]", - inputRow - ); - throw new ParseException(errorMsg); - } - - final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); - if (!optInterval.isPresent()) { - buildSegmentsMeters.incrementThrownAway(); continue; } + Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + @SuppressWarnings("OptionalGetWithoutIsPresent") // always present via IndexTaskInputRowIteratorBuilder final Interval interval = optInterval.get(); final String sequenceName = segmentAllocator.getSequenceName(interval, inputRow); final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 51b44f18d117..cadcb1cb5d73 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -53,6 +53,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; +import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; @@ -802,7 +803,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( // We use the timeChunk lock and don't have to ask the overlord to create segmentIds. // Instead, a local allocator is used. if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { - return new CachingLocalSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); + return new DefaultCachingLocalSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); } else { return new LocalSegmentAllocator(toolbox, getId(), getDataSource(), dataSchema.getGranularitySpec()); } @@ -885,7 +886,8 @@ private TaskStatus generateAndPublishSegments( buildSegmentsSavedParseExceptions, tuningConfig.isLogParseExceptions(), tuningConfig.getMaxParseExceptions(), - pushTimeout + pushTimeout, + new DefaultIndexTaskInputRowIteratorBuilder() ); firehoseProcessor.process( dataSchema, @@ -968,26 +970,12 @@ static class ShardSpecs this.map = map; } - /** - * Return the underlying map. - * - * @return a map of intervals to shardSpecs - */ - Map> getMap() - { - return map; - } - - Set getIntervals() - { - return map.keySet(); - } - /** * Return a shardSpec for the given interval and input row. * * @param interval interval for shardSpec * @param row input row + * * @return a shardSpec */ ShardSpec getShardSpec(Interval interval, InputRow row) @@ -1109,7 +1097,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - public static IndexTuningConfig createDefault() + static IndexTuningConfig createDefault() { return new IndexTuningConfig(); } @@ -1142,8 +1130,8 @@ private static PartitionsSpec getDefaultPartitionsSpec( } } else { if (forceGuaranteedRollup) { - if (!(partitionsSpec instanceof HashedPartitionsSpec)) { - throw new ISE("HashedPartitionsSpec must be used for perfect rollup"); + if (!partitionsSpec.isForceGuaranteedRollupCompatibleType()) { + throw new ISE(partitionsSpec.getClass().getSimpleName() + " cannot be used for perfect rollup"); } } else { if (!(partitionsSpec instanceof DynamicPartitionsSpec)) { @@ -1364,7 +1352,7 @@ public int getMaxPendingPersists() */ @Deprecated @JsonProperty - public boolean isBuildV9Directly() + public static boolean isBuildV9Directly() { return true; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 58612ce523ed..089dccb2c82a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -26,8 +26,8 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; -import org.apache.druid.indexing.common.task.batch.parallel.PartialSegmentGenerateTask; -import org.apache.druid.indexing.common.task.batch.parallel.PartialSegmentMergeTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; @@ -56,8 +56,8 @@ @JsonSubTypes.Type(name = ParallelIndexSupervisorTask.TYPE, value = ParallelIndexSupervisorTask.class), @JsonSubTypes.Type(name = SinglePhaseSubTask.TYPE, value = SinglePhaseSubTask.class), @JsonSubTypes.Type(name = "index_sub", value = SinglePhaseSubTask.class), // for backward compatibility - @JsonSubTypes.Type(name = PartialSegmentGenerateTask.TYPE, value = PartialSegmentGenerateTask.class), - @JsonSubTypes.Type(name = PartialSegmentMergeTask.TYPE, value = PartialSegmentMergeTask.class), + @JsonSubTypes.Type(name = PartialHashSegmentGenerateTask.TYPE, value = PartialHashSegmentGenerateTask.class), + @JsonSubTypes.Type(name = PartialHashSegmentMergeTask.TYPE, value = PartialHashSegmentMergeTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), @JsonSubTypes.Type(name = "index_realtime_appenderator", value = AppenderatorDriverRealtimeIndexTask.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java similarity index 63% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java index edb00aa100d3..0cbe7b8634df 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java @@ -19,28 +19,27 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.Task; import java.io.IOException; import java.util.Iterator; import java.util.Map; /** - * {@link ParallelIndexTaskRunner} for the phase to create partitioned segments in multi-phase parallel indexing. - * - * @see PartialSegmentMergeParallelIndexTaskRunner + * Base class for different implementations of {@link ParallelIndexTaskRunner} that operate on + * {@link org.apache.druid.data.input.Firehose} splits. */ -class PartialSegmentGenerateParallelIndexTaskRunner - extends ParallelIndexPhaseRunner +abstract class FirehoseSplitParallelIndexTaskRunner + extends ParallelIndexPhaseRunner { private final ParallelIndexIngestionSpec ingestionSchema; private final FiniteFirehoseFactory baseFirehoseFactory; - PartialSegmentGenerateParallelIndexTaskRunner( + FirehoseSplitParallelIndexTaskRunner( TaskToolbox toolbox, String taskId, String groupId, @@ -62,36 +61,18 @@ class PartialSegmentGenerateParallelIndexTaskRunner } @Override - public String getName() - { - return PartialSegmentGenerateTask.TYPE; - } - - @Override - Iterator> subTaskSpecIterator() throws IOException + Iterator> subTaskSpecIterator() throws IOException { return baseFirehoseFactory.getSplits(getTuningConfig().getSplitHintSpec()).map(this::newTaskSpec).iterator(); } @Override - int getTotalNumSubTasks() throws IOException + final int getTotalNumSubTasks() throws IOException { return baseFirehoseFactory.getNumSplits(getTuningConfig().getSplitHintSpec()); } - @VisibleForTesting - ParallelIndexIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - - @VisibleForTesting - FiniteFirehoseFactory getBaseFirehoseFactory() - { - return baseFirehoseFactory; - } - - SubTaskSpec newTaskSpec(InputSplit split) + final SubTaskSpec newTaskSpec(InputSplit split) { final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), @@ -101,30 +82,25 @@ SubTaskSpec newTaskSpec(InputSplit split) ), ingestionSchema.getTuningConfig() ); - return new SubTaskSpec( + + return createSubTaskSpec( getTaskId() + "_" + getAndIncrementNextSpecId(), getGroupId(), getTaskId(), getContext(), - split - ) - { - @Override - public PartialSegmentGenerateTask newSubTask(int numAttempts) - { - return new PartialSegmentGenerateTask( - null, - getGroupId(), - null, - getSupervisorTaskId(), - numAttempts, - subTaskIngestionSpec, - getContext(), - getIndexingServiceClient(), - null, - null - ); - } - }; + split, + subTaskIngestionSpec, + getIndexingServiceClient() + ); } + + abstract SubTaskSpec createSubTaskSpec( + String id, + String groupId, + String supervisorTaskId, + Map context, + InputSplit split, + ParallelIndexIngestionSpec subTaskIngestionSpec, + IndexingServiceClient indexingServiceClient + ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java index d1fd0000d13f..9cf72ddbd6ee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java @@ -30,17 +30,17 @@ * This report is collected by {@link ParallelIndexSupervisorTask} and * used to generate {@link PartialSegmentMergeIOConfig}. */ -public class GeneratedPartitionsReport implements SubTaskReport +public class GeneratedPartitionsReport implements SubTaskReport { public static final String TYPE = "generated_partitions"; private final String taskId; - private final List partitionStats; + private final List partitionStats; @JsonCreator public GeneratedPartitionsReport( @JsonProperty("taskId") String taskId, - @JsonProperty("partitionStats") List partitionStats + @JsonProperty("partitionStats") List partitionStats ) { this.taskId = taskId; @@ -55,7 +55,7 @@ public String getTaskId() } @JsonProperty - public List getPartitionStats() + public List getPartitionStats() { return partitionStats; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java new file mode 100644 index 000000000000..ae766793f821 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; + +/** + * This class represents the intermediary data server where the partition of {@link #interval} and {@link #partitionId} + * is stored. + */ +public class HashPartitionLocation extends PartitionLocation +{ + private final int partitionId; + + @JsonCreator + public HashPartitionLocation( + @JsonProperty("host") String host, + @JsonProperty("port") int port, + @JsonProperty("useHttps") boolean useHttps, + @JsonProperty("subTaskId") String subTaskId, + @JsonProperty("interval") Interval interval, + @JsonProperty("partitionId") int partitionId + ) + { + super(host, port, useHttps, subTaskId, interval, partitionId); + this.partitionId = partitionId; + } + + @JsonProperty + @Override + public int getPartitionId() + { + return partitionId; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java new file mode 100644 index 000000000000..21019abe0fe7 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Statistics about a partition created by {@link PartialHashSegmentGenerateTask}. Each partition is a set of data + * of the same time chunk (primary partition key) and the same partitionId (secondary partition key). This class + * holds the statistics of a single partition created by a task. + */ +public class HashPartitionStat extends PartitionStat +{ + // Secondary partition key + private final int partitionId; + + @JsonCreator + public HashPartitionStat( + @JsonProperty("taskExecutorHost") String taskExecutorHost, + @JsonProperty("taskExecutorPort") int taskExecutorPort, + @JsonProperty("useHttps") boolean useHttps, + @JsonProperty("interval") Interval interval, + @JsonProperty("partitionId") int partitionId, + @JsonProperty("numRows") @Nullable Integer numRows, + @JsonProperty("sizeBytes") @Nullable Long sizeBytes + ) + { + super(taskExecutorHost, taskExecutorPort, useHttps, interval, numRows, sizeBytes); + this.partitionId = partitionId; + } + + @JsonProperty + @Override + public int getPartitionId() + { + return partitionId; + } + + @JsonIgnore + @Override + Integer getSecondaryPartition() + { + return partitionId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + HashPartitionStat that = (HashPartitionStat) o; + return partitionId == that.partitionId; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), partitionId); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java index e3a7632b48a2..48e263db5a2e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.ForwardingIterator; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -112,6 +113,44 @@ public abstract class ParallelIndexPhaseRunner> + { + private final Iterator> delegate; + + UniqueSubTaskIdDecorator(Iterator> subTaskSpecIterator) + { + delegate = subTaskSpecIterator; + } + + @Override + protected Iterator> delegate() + { + return delegate; + } + + @Override + public SubTaskSpec next() + { + ensureUniqueSubtaskId(); + return super.next(); + } + + private void ensureUniqueSubtaskId() + { + try { + // Ensure each subtask has a different id (which includes the current time -- see AbstractTask.getOrMakeId()) + Thread.sleep(1); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + @Override public TaskState run() throws Exception { @@ -120,7 +159,7 @@ public TaskState run() throws Exception return TaskState.SUCCESS; } - final Iterator> subTaskSpecIterator = subTaskSpecIterator(); + final Iterator> subTaskSpecIterator = new UniqueSubTaskIdDecorator(subTaskSpecIterator()); final long taskStatusCheckingPeriod = tuningConfig.getTaskStatusCheckPeriodMs(); taskMonitor = new TaskMonitor<>( @@ -161,7 +200,7 @@ public TaskState run() throws Exception if (!subTaskSpecIterator.hasNext()) { // We have no more subTasks to run - if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) { + if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.isEmpty()) { subTaskScheduleAndMonitorStopped = true; if (taskMonitor.isSucceeded()) { // Succeeded diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 265d38dd6053..39760dc1dea1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -32,6 +32,8 @@ import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.Counters; import org.apache.druid.indexing.common.TaskLock; @@ -100,6 +102,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; @@ -185,9 +188,7 @@ public ParallelIndexSupervisorTask( } if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { - if (ingestionSchema.getTuningConfig().getNumShards() == null) { - throw new ISE("forceGuaranteedRollup is set but numShards is missing in partitionsSpec"); - } + checkPartitionsSpecForForceGuaranteedRollup(ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec()); if (ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals().isEmpty()) { throw new ISE("forceGuaranteedRollup is set but intervals is missing in granularitySpec"); @@ -208,7 +209,15 @@ public ParallelIndexSupervisorTask( if (missingIntervalsInOverwriteMode) { addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } + } + private static void checkPartitionsSpecForForceGuaranteedRollup(PartitionsSpec partitionsSpec) + { + if (!partitionsSpec.isForceGuaranteedRollupCompatible()) { + String incompatibiltyMsg = partitionsSpec.getForceGuaranteedRollupIncompatiblityReason(); + String msg = "forceGuaranteedRollup is incompatible with partitionsSpec: " + incompatibiltyMsg; + throw new ISE(msg); + } } @Override @@ -260,7 +269,7 @@ private ParallelIndexTaskRunner } } - private TaskState runNextPhase(@Nullable ParallelIndexTaskRunner nextPhaseRunner) throws Exception + private static TaskState runNextPhase(@Nullable ParallelIndexTaskRunner nextPhaseRunner) throws Exception { if (nextPhaseRunner == null) { LOG.info("Task is asked to stop. Finish as failed"); @@ -284,9 +293,9 @@ SinglePhaseParallelIndexTaskRunner createSinglePhaseTaskRunner(TaskToolbox toolb } @VisibleForTesting - public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerateRunner(TaskToolbox toolbox) + PartialHashSegmentGenerateParallelIndexTaskRunner createPartialHashSegmentGenerateRunner(TaskToolbox toolbox) { - return new PartialSegmentGenerateParallelIndexTaskRunner( + return new PartialHashSegmentGenerateParallelIndexTaskRunner( toolbox, getId(), getGroupId(), @@ -297,12 +306,12 @@ public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerat } @VisibleForTesting - public PartialSegmentMergeParallelIndexTaskRunner createPartialSegmentMergeRunner( + PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner( TaskToolbox toolbox, - List ioConfigs + List> ioConfigs ) { - return new PartialSegmentMergeParallelIndexTaskRunner( + return new PartialHashSegmentMergeParallelIndexTaskRunner( toolbox, getId(), getGroupId(), @@ -462,17 +471,31 @@ private TaskStatus runSinglePhaseParallel(TaskToolbox toolbox) throws Exception * * - In the first phase, each task partitions input data and stores those partitions in local storage. * - The partition is created based on the segment granularity (primary partition key) and the partition dimension - * values in {@link org.apache.druid.indexer.partitions.PartitionsSpec} (secondary partition key). + * values in {@link PartitionsSpec} (secondary partition key). * - Partitioned data is maintained by {@link org.apache.druid.indexing.worker.IntermediaryDataManager}. * - In the second phase, each task reads partitioned data from the intermediary data server (middleManager * or indexer) and merges them to create the final segments. */ private TaskStatus runMultiPhaseParallel(TaskToolbox toolbox) throws Exception + { + if (useHashPartitions()) { + return runHashPartitionMultiPhaseParallel(toolbox); + } else { + throw new UnsupportedOperationException("hash partition required"); + } + } + + private boolean useHashPartitions() + { + return (ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec() instanceof HashedPartitionsSpec); + } + + private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throws Exception { // 1. Partial segment generation phase - final ParallelIndexTaskRunner indexingRunner = createRunner( + ParallelIndexTaskRunner> indexingRunner = createRunner( toolbox, - this::createPartialSegmentGenerateRunner + this::createPartialHashSegmentGenerateRunner ); TaskState state = runNextPhase(indexingRunner); @@ -483,18 +506,16 @@ private TaskStatus runMultiPhaseParallel(TaskToolbox toolbox) throws Exception // 2. Partial segment merge phase // partition (interval, partitionId) -> partition locations - //noinspection ConstantConditions - Map, List> partitionToLocations = groupPartitionLocationsPerPartition( - indexingRunner.getReports() - ); - final List ioConfigs = createMergeIOConfigs( + Map, List> partitionToLocations = + groupHashPartitionLocationsPerPartition(indexingRunner.getReports()); + final List> ioConfigs = createMergeIOConfigs( ingestionSchema.getTuningConfig().getTotalNumMergeTasks(), partitionToLocations ); - final ParallelIndexTaskRunner mergeRunner = createRunner( + final ParallelIndexTaskRunner mergeRunner = createRunner( toolbox, - tb -> createPartialSegmentMergeRunner(tb, ioConfigs) + tb -> createPartialHashSegmentMergeRunner(tb, ioConfigs) ); state = runNextPhase(mergeRunner); if (state.isSuccess()) { @@ -505,40 +526,50 @@ private TaskStatus runMultiPhaseParallel(TaskToolbox toolbox) throws Exception return TaskStatus.fromCode(getId(), state); } - private static Map, List> groupPartitionLocationsPerPartition( - // subTaskId -> report - Map reports + private static Map, List> groupHashPartitionLocationsPerPartition( + Map> subTaskIdToReport ) { - // partition (interval, partitionId) -> partition locations - final Map, List> partitionToLocations = new HashMap<>(); - for (Entry entry : reports.entrySet()) { - final String subTaskId = entry.getKey(); - final GeneratedPartitionsReport report = entry.getValue(); - for (PartitionStat partitionStat : report.getPartitionStats()) { - final List locationsOfSamePartition = partitionToLocations.computeIfAbsent( - Pair.of(partitionStat.getInterval(), partitionStat.getPartitionId()), - k -> new ArrayList<>() - ); - locationsOfSamePartition.add( - new PartitionLocation( + BiFunction createPartitionLocationFunction = + (subtaskId, partitionStat) -> + new HashPartitionLocation( partitionStat.getTaskExecutorHost(), partitionStat.getTaskExecutorPort(), partitionStat.isUseHttps(), - subTaskId, + subtaskId, partitionStat.getInterval(), - partitionStat.getPartitionId() - ) + partitionStat.getSecondaryPartition() + ); + + return groupPartitionLocationsPerPartition(subTaskIdToReport, createPartitionLocationFunction); + } + + private static + Map, List> groupPartitionLocationsPerPartition( + Map> subTaskIdToReport, + BiFunction createPartitionLocationFunction + ) + { + // partition (interval, partitionId) -> partition locations + final Map, List> partitionToLocations = new HashMap<>(); + for (Entry> entry : subTaskIdToReport.entrySet()) { + final String subTaskId = entry.getKey(); + final GeneratedPartitionsReport report = entry.getValue(); + for (S partitionStat : report.getPartitionStats()) { + final List locationsOfSamePartition = partitionToLocations.computeIfAbsent( + Pair.of(partitionStat.getInterval(), partitionStat.getPartitionId()), + k -> new ArrayList<>() ); + locationsOfSamePartition.add(createPartitionLocationFunction.apply(subTaskId, partitionStat)); } } return partitionToLocations; } - private static List createMergeIOConfigs( + private static List> createMergeIOConfigs( int totalNumMergeTasks, - Map, List> partitionToLocations + Map, List> partitionToLocations ) { final int numMergeTasks = Math.min(totalNumMergeTasks, partitionToLocations.size()); @@ -550,28 +581,28 @@ private static List createMergeIOConfigs( ); // Randomly shuffle partitionIds to evenly distribute partitions of potentially different sizes // This will be improved once we collect partition stats properly. - // See PartitionStat in GeneratedPartitionsReport. + // See HashPartitionStat in GeneratedPartitionsReport. final List> partitions = new ArrayList<>(partitionToLocations.keySet()); Collections.shuffle(partitions, ThreadLocalRandom.current()); final int numPartitionsPerTask = (int) Math.round(partitions.size() / (double) numMergeTasks); - final List assignedPartitionLocations = new ArrayList<>(numMergeTasks); + final List> assignedPartitionLocations = new ArrayList<>(numMergeTasks); for (int i = 0; i < numMergeTasks - 1; i++) { - final List assignedToSameTask = partitions + final List assignedToSameTask = partitions .subList(i * numPartitionsPerTask, (i + 1) * numPartitionsPerTask) .stream() .flatMap(intervalAndPartitionId -> partitionToLocations.get(intervalAndPartitionId).stream()) .collect(Collectors.toList()); - assignedPartitionLocations.add(new PartialSegmentMergeIOConfig(assignedToSameTask)); + assignedPartitionLocations.add(new PartialSegmentMergeIOConfig(assignedToSameTask)); } // The last task is assigned all remaining partitions. - final List assignedToSameTask = partitions + final List assignedToSameTask = partitions .subList((numMergeTasks - 1) * numPartitionsPerTask, partitions.size()) .stream() .flatMap(intervalAndPartitionId -> partitionToLocations.get(intervalAndPartitionId).stream()) .collect(Collectors.toList()); - assignedPartitionLocations.add(new PartialSegmentMergeIOConfig(assignedToSameTask)); + assignedPartitionLocations.add(new PartialSegmentMergeIOConfig<>(assignedToSameTask)); return assignedPartitionLocations; } @@ -779,7 +810,9 @@ private static String findVersion(Map versions, Interval inter } /** - * {@link SinglePhaseSubTask}s call this API to report the segments they generated and pushed. + * Worker tasks spawned by the supervisor call this API to report the segments they generated and pushed. + * + * @see ParallelIndexSupervisorTaskClient#report(String, SubTaskReport) */ @POST @Path("/report") diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java index 8100560f1696..c4ebef5aea86 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java @@ -38,7 +38,7 @@ * uses {@link SinglePhaseParallelIndexTaskRunner} for it. * * For perfect rollup, parallel indexing is executed in multiple phases. The supervisor task currently uses - * {@link PartialSegmentGenerateParallelIndexTaskRunner} and {@link PartialSegmentMergeParallelIndexTaskRunner}, + * {@link PartialHashSegmentGenerateParallelIndexTaskRunner} and {@link PartialHashSegmentMergeParallelIndexTaskRunner}, * and can use more runners in the future. */ public interface ParallelIndexTaskRunner diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java index 552c4048a80d..dbfc73d89b3f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java @@ -58,14 +58,14 @@ public class ParallelIndexTuningConfig extends IndexTuningConfig /** * Max number of segments to merge at the same time. - * Used only by {@link PartialSegmentMergeTask}. + * Used only by {@link PartialHashSegmentMergeTask}. * This configuration was temporarily added to avoid using too much memory while merging segments, * and will be removed once {@link org.apache.druid.segment.IndexMerger} is improved to not use much memory. */ private final int maxNumSegmentsToMerge; /** - * Total number of tasks for partial segment merge (that is, number of {@link PartialSegmentMergeTask}s). + * Total number of tasks for partial segment merge (that is, number of {@link PartialHashSegmentMergeTask}s). * Used only when this task runs with shuffle. */ private final int totalNumMergeTasks; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java new file mode 100644 index 000000000000..32fb0b1ee572 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; + +import java.util.Map; + +/** + * {@link ParallelIndexTaskRunner} for the phase to create hash partitioned segments in multi-phase parallel indexing. + * + * @see PartialHashSegmentMergeParallelIndexTaskRunner + */ +class PartialHashSegmentGenerateParallelIndexTaskRunner + extends FirehoseSplitParallelIndexTaskRunner> +{ + // For tests + private final IndexTaskClientFactory taskClientFactory; + private final AppenderatorsManager appenderatorsManager; + + PartialHashSegmentGenerateParallelIndexTaskRunner( + TaskToolbox toolbox, + String taskId, + String groupId, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + this(toolbox, taskId, groupId, ingestionSchema, context, indexingServiceClient, null, null); + } + + @VisibleForTesting + PartialHashSegmentGenerateParallelIndexTaskRunner( + TaskToolbox toolbox, + String taskId, + String groupId, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient, + IndexTaskClientFactory taskClientFactory, + AppenderatorsManager appenderatorsManager + ) + { + super(toolbox, taskId, groupId, ingestionSchema, context, indexingServiceClient); + this.taskClientFactory = taskClientFactory; + this.appenderatorsManager = appenderatorsManager; + } + + @Override + public String getName() + { + return PartialHashSegmentGenerateTask.TYPE; + } + + @Override + SubTaskSpec createSubTaskSpec( + String id, + String groupId, + String supervisorTaskId, + Map context, + InputSplit split, + ParallelIndexIngestionSpec subTaskIngestionSpec, + IndexingServiceClient indexingServiceClient + ) + { + return new SubTaskSpec( + id, + groupId, + supervisorTaskId, + context, + split + ) + { + @Override + public PartialHashSegmentGenerateTask newSubTask(int numAttempts) + { + return new PartialHashSegmentGenerateTask( + null, + groupId, + null, + supervisorTaskId, + numAttempts, + subTaskIngestionSpec, + context, + indexingServiceClient, + taskClientFactory, + appenderatorsManager + ); + } + }; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java new file mode 100644 index 000000000000..17c3db15fd0d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.task.DefaultCachingLocalSegmentAllocator; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; +import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Map; + +/** + * The worker task of {@link PartialHashSegmentGenerateParallelIndexTaskRunner}. This task partitions input data by + * hashing the segment granularity and partition dimensions in {@link + * org.apache.druid.indexer.partitions.PartitionsSpec}. Partitioned segments are stored in local storage using {@link + * ShuffleDataSegmentPusher}. + */ +public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask +{ + public static final String TYPE = "partial_index_generate"; + private static final String PROP_SPEC = "spec"; + + private final int numAttempts; + private final ParallelIndexIngestionSpec ingestionSchema; + private final String supervisorTaskId; + + @JsonCreator + public PartialHashSegmentGenerateTask( + // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask + @JsonProperty("id") @Nullable 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(PROP_SPEC) final ParallelIndexIngestionSpec ingestionSchema, + @JsonProperty("context") final Map context, + @JacksonInject IndexingServiceClient indexingServiceClient, + @JacksonInject IndexTaskClientFactory taskClientFactory, + @JacksonInject AppenderatorsManager appenderatorsManager + ) + { + super( + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + groupId, + taskResource, + supervisorTaskId, + ingestionSchema, + context, + indexingServiceClient, + taskClientFactory, + appenderatorsManager, + new DefaultIndexTaskInputRowIteratorBuilder() + ); + + this.numAttempts = numAttempts; + this.ingestionSchema = ingestionSchema; + this.supervisorTaskId = supervisorTaskId; + } + + @JsonProperty + public int getNumAttempts() + { + return numAttempts; + } + + @JsonProperty(PROP_SPEC) + public ParallelIndexIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @JsonProperty + public String getSupervisorTaskId() + { + return supervisorTaskId; + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return tryTimeChunkLock( + taskActionClient, + getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals() + ); + } + + @Override + IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException + { + return new DefaultCachingLocalSegmentAllocator( + toolbox, + getId(), + getDataSource(), + createShardSpecs() + ); + } + + @Override + HashPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment) + { + return new HashPartitionStat( + toolbox.getTaskExecutorNode().getHost(), + toolbox.getTaskExecutorNode().getPortToUse(), + toolbox.getTaskExecutorNode().isEnableTlsPort(), + segment.getInterval(), + segment.getShardSpec().getPartitionNum(), + null, // numRows is not supported yet + null // sizeBytes is not supported yet + ); + } + + private Map> createShardSpecs() + { + GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); + final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec(); + + return createShardSpecWithoutInputScan( + granularitySpec, + ingestionSchema.getIOConfig(), + tuningConfig, + partitionsSpec + ); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java similarity index 67% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeParallelIndexTaskRunner.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java index e0a06a55aff4..a5e45abe754e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java @@ -30,22 +30,22 @@ import java.util.Map; /** - * {@link ParallelIndexTaskRunner} for the phase to merge partitioned segments in multi-phase parallel indexing. + * {@link ParallelIndexTaskRunner} for the phase to merge hash partitioned segments in multi-phase parallel indexing. * - * @see PartialSegmentGenerateParallelIndexTaskRunner + * @see PartialHashSegmentGenerateParallelIndexTaskRunner */ -class PartialSegmentMergeParallelIndexTaskRunner - extends ParallelIndexPhaseRunner +class PartialHashSegmentMergeParallelIndexTaskRunner + extends ParallelIndexPhaseRunner { private final DataSchema dataSchema; - private final List mergeIOConfigs; + private final List> mergeIOConfigs; - PartialSegmentMergeParallelIndexTaskRunner( + PartialHashSegmentMergeParallelIndexTaskRunner( TaskToolbox toolbox, String taskId, String groupId, DataSchema dataSchema, - List mergeIOConfigs, + List> mergeIOConfigs, ParallelIndexTuningConfig tuningConfig, Map context, IndexingServiceClient indexingServiceClient @@ -60,11 +60,11 @@ class PartialSegmentMergeParallelIndexTaskRunner @Override public String getName() { - return PartialSegmentMergeTask.TYPE; + return PartialHashSegmentMergeTask.TYPE; } @Override - Iterator> subTaskSpecIterator() + Iterator> subTaskSpecIterator() { return mergeIOConfigs.stream().map(this::newTaskSpec).iterator(); } @@ -76,14 +76,15 @@ int getTotalNumSubTasks() } @VisibleForTesting - SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) + SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) { - final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec( - dataSchema, - ioConfig, - getTuningConfig() - ); - return new SubTaskSpec( + final PartialSegmentMergeIngestionSpec ingestionSpec = + new PartialSegmentMergeIngestionSpec<>( + dataSchema, + ioConfig, + getTuningConfig() + ); + return new SubTaskSpec( getTaskId() + "_" + getAndIncrementNextSpecId(), getGroupId(), getTaskId(), @@ -92,9 +93,9 @@ SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioC ) { @Override - public PartialSegmentMergeTask newSubTask(int numAttempts) + public PartialHashSegmentMergeTask newSubTask(int numAttempts) { - return new PartialSegmentMergeTask( + return new PartialHashSegmentMergeTask( null, getGroupId(), null, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java new file mode 100644 index 000000000000..c15af6474423 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.guice.annotations.EscalatedClient; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Map; + +/** + * The worker task of {@link PartialHashSegmentMergeParallelIndexTaskRunner}. This task reads partitioned segments + * created by {@link PartialHashSegmentGenerateTask}s, merges them, and pushes to deep storage. The pushed segments are + * reported to {@link PartialHashSegmentMergeParallelIndexTaskRunner}. + */ + +public class PartialHashSegmentMergeTask + extends PartialSegmentMergeTask +{ + public static final String TYPE = "partial_index_merge"; + + private final HashedPartitionsSpec partitionsSpec; + private final PartialSegmentMergeIngestionSpec ingestionSchema; + + @JsonCreator + public PartialHashSegmentMergeTask( + // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask + @JsonProperty("id") @Nullable 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 PartialSegmentMergeIngestionSpec ingestionSchema, + @JsonProperty("context") final Map context, + @JacksonInject IndexingServiceClient indexingServiceClient, + @JacksonInject IndexTaskClientFactory taskClientFactory, + @JacksonInject @EscalatedClient HttpClient shuffleClient + ) + { + super( + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + groupId, + taskResource, + supervisorTaskId, + ingestionSchema.getDataSchema(), + ingestionSchema.getIOConfig(), + ingestionSchema.getTuningConfig(), + numAttempts, + context, + indexingServiceClient, + taskClientFactory, + shuffleClient + ); + + this.ingestionSchema = ingestionSchema; + + PartitionsSpec inputPartitionsSpec = ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec(); + Preconditions.checkArgument(inputPartitionsSpec instanceof HashedPartitionsSpec, "hashed partitionsSpec required"); + partitionsSpec = (HashedPartitionsSpec) inputPartitionsSpec; + Preconditions.checkNotNull(partitionsSpec.getNumShards(), "hashed partitionsSpec numShards required"); + } + + @JsonProperty("spec") + private PartialSegmentMergeIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + HashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionNum) + { + return new HashBasedNumberedShardSpec( + partitionNum, + Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), + partitionsSpec.getPartitionDimensions(), + toolbox.getObjectMapper() + ); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index dd35a43b5006..58d433500857 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -19,36 +19,26 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; import org.apache.commons.io.FileUtils; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMeters; -import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.BatchAppenderators; -import org.apache.druid.indexing.common.task.CachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.FiniteFirehoseProcessor; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.RealtimeMetricsMonitor; @@ -57,10 +47,7 @@ import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.ShardSpecFactory; -import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -71,140 +58,49 @@ import java.util.stream.Collectors; /** - * The worker task of {@link PartialSegmentGenerateParallelIndexTaskRunner}. This task partitions input data by - * the segment granularity and partition dimensions in {@link org.apache.druid.indexer.partitions.PartitionsSpec}. - * Partitioned segments are stored in local storage using {@link ShuffleDataSegmentPusher}. + * Base class for parallel indexing perfect rollup worker partial segment generate tasks. */ -public class PartialSegmentGenerateTask extends AbstractBatchIndexTask +abstract class PartialSegmentGenerateTask extends PerfectRollupWorkerTask { - public static final String TYPE = "partial_index_generate"; - - private final int numAttempts; private final ParallelIndexIngestionSpec ingestionSchema; private final String supervisorTaskId; private final IndexingServiceClient indexingServiceClient; private final IndexTaskClientFactory taskClientFactory; private final AppenderatorsManager appenderatorsManager; - - @JsonCreator - public PartialSegmentGenerateTask( - // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask - @JsonProperty("id") @Nullable 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 ParallelIndexIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context, - @JacksonInject IndexingServiceClient indexingServiceClient, - @JacksonInject IndexTaskClientFactory taskClientFactory, - @JacksonInject AppenderatorsManager appenderatorsManager + private final IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder; + + PartialSegmentGenerateTask( + String id, + String groupId, + TaskResource taskResource, + String supervisorTaskId, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient, + IndexTaskClientFactory taskClientFactory, + AppenderatorsManager appenderatorsManager, + IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder ) { super( - getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + id, groupId, taskResource, - ingestionSchema.getDataSchema().getDataSource(), + ingestionSchema.getDataSchema(), + ingestionSchema.getTuningConfig(), context ); - Preconditions.checkArgument( - ingestionSchema.getTuningConfig().isForceGuaranteedRollup(), - "forceGuaranteedRollup must be set" - ); - Preconditions.checkArgument( - ingestionSchema.getTuningConfig().getPartitionsSpec() == null - || ingestionSchema.getTuningConfig().getPartitionsSpec() instanceof HashedPartitionsSpec, - "Please use hashed_partitions for perfect rollup" - ); - Preconditions.checkArgument( - !ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals().isEmpty(), - "Missing intervals in granularitySpec" - ); - - this.numAttempts = numAttempts; this.ingestionSchema = ingestionSchema; this.supervisorTaskId = supervisorTaskId; this.indexingServiceClient = indexingServiceClient; this.taskClientFactory = taskClientFactory; this.appenderatorsManager = appenderatorsManager; - } - - @JsonProperty - public int getNumAttempts() - { - return numAttempts; - } - - @JsonProperty("spec") - public ParallelIndexIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - - @JsonProperty - public String getSupervisorTaskId() - { - return supervisorTaskId; - } - - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); - } - - @Override - public String getType() - { - return TYPE; - } - - @Override - public boolean requireLockExistingSegments() - { - return true; + this.inputRowIteratorBuilder = inputRowIteratorBuilder; } @Override - public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) - { - throw new UnsupportedOperationException( - "This method should be never called because ParallelIndexGeneratingTask always uses timeChunk locking" - + " but this method is supposed to be called only with segment locking." - ); - } - - @Override - public boolean isPerfectRollup() - { - return true; - } - - @Nullable - @Override - public Granularity getSegmentGranularity() - { - final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - if (granularitySpec instanceof ArbitraryGranularitySpec) { - return null; - } else { - return granularitySpec.getSegmentGranularity(); - } - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) throws Exception - { - return tryTimeChunkLock( - taskActionClient, - getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals() - ); - } - - @Override - public TaskStatus runTask(TaskToolbox toolbox) throws Exception + public final TaskStatus runTask(TaskToolbox toolbox) throws Exception { final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); @@ -221,23 +117,24 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); final List segments = generateSegments(toolbox, firehoseFactory, firehoseTempDir); - final List partitionStats = segments - .stream() - .map(segment -> new PartitionStat( - toolbox.getTaskExecutorNode().getHost(), - toolbox.getTaskExecutorNode().getPortToUse(), - toolbox.getTaskExecutorNode().isEnableTlsPort(), - segment.getInterval(), - segment.getShardSpec().getPartitionNum(), - null, // numRows is not supported yet - null // sizeBytes is not supported yet - )) - .collect(Collectors.toList()); - taskClient.report(supervisorTaskId, new GeneratedPartitionsReport(getId(), partitionStats)); + final List partitionStats = segments.stream() + .map(segment -> createPartitionStat(toolbox, segment)) + .collect(Collectors.toList()); + taskClient.report(supervisorTaskId, new GeneratedPartitionsReport<>(getId(), partitionStats)); return TaskStatus.success(getId()); } + /** + * @return {@link IndexTaskSegmentAllocator} suitable for the desired segment partitioning strategy. + */ + abstract IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException; + + /** + * @return {@link PartitionStat} suitable for the desired segment partitioning strategy. + */ + abstract T createPartitionStat(TaskToolbox toolbox, DataSegment segment); + private List generateSegments( final TaskToolbox toolbox, final FirehoseFactory firehoseFactory, @@ -245,7 +142,6 @@ private List generateSegments( ) throws IOException, InterruptedException, ExecutionException, TimeoutException { final DataSchema dataSchema = ingestionSchema.getDataSchema(); - final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); final FireDepartment fireDepartmentForMetrics = new FireDepartment( dataSchema, new RealtimeIOConfig(null, null), @@ -264,22 +160,10 @@ private List generateSegments( } final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); - final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec(); + final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec(); final long pushTimeout = tuningConfig.getPushTimeout(); - final Map> shardSpecs = createShardSpecWithoutInputScan( - granularitySpec, - ingestionSchema.getIOConfig(), - tuningConfig, - partitionsSpec - ); - - final IndexTaskSegmentAllocator segmentAllocator = new CachingLocalSegmentAllocator( - toolbox, - getId(), - getDataSource(), - shardSpecs - ); + final IndexTaskSegmentAllocator segmentAllocator = createSegmentAllocator(toolbox); final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), @@ -300,7 +184,8 @@ private List generateSegments( null, tuningConfig.isLogParseExceptions(), tuningConfig.getMaxParseExceptions(), - pushTimeout + pushTimeout, + inputRowIteratorBuilder ); final SegmentsAndMetadata pushed = firehoseProcessor.process( dataSchema, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java index 4b5b4c3689a1..4d259f60acf7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java @@ -27,13 +27,13 @@ import java.util.List; -@JsonTypeName(PartialSegmentMergeTask.TYPE) -public class PartialSegmentMergeIOConfig implements IOConfig +@JsonTypeName(PartialHashSegmentMergeTask.TYPE) +public class PartialSegmentMergeIOConfig implements IOConfig { - private final List partitionLocations; + private final List partitionLocations; @JsonCreator - public PartialSegmentMergeIOConfig(@JsonProperty("partitionLocations") List partitionLocations) + public PartialSegmentMergeIOConfig(@JsonProperty("partitionLocations") List partitionLocations) { Preconditions.checkState( partitionLocations != null && !partitionLocations.isEmpty(), @@ -43,7 +43,7 @@ public PartialSegmentMergeIOConfig(@JsonProperty("partitionLocations") List getPartitionLocations() + public List getPartitionLocations() { return partitionLocations; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java index e4c6e260c949..67d4868c28e5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java @@ -24,13 +24,13 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; -public class PartialSegmentMergeIngestionSpec - extends IngestionSpec +public class PartialSegmentMergeIngestionSpec + extends IngestionSpec, ParallelIndexTuningConfig> { @JsonCreator public PartialSegmentMergeIngestionSpec( @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") PartialSegmentMergeIOConfig ioConfig, + @JsonProperty("ioConfig") PartialSegmentMergeIOConfig ioConfig, @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig ) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 2a3953468da9..eb6384220fcf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; @@ -29,15 +28,12 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; import org.apache.commons.io.FileUtils; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.guice.annotations.EscalatedClient; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; @@ -45,7 +41,6 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.HttpClient; @@ -56,11 +51,10 @@ import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.utils.CompressionUtils; import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Interval; @@ -83,70 +77,55 @@ import java.util.stream.Collectors; /** - * The worker task of {@link PartialSegmentMergeParallelIndexTaskRunner}. This task reads partitioned segments created - * by {@link PartialSegmentGenerateTask}s, merges them, and pushes to the deep storage. The pushed segments are reported - * to {@link PartialSegmentMergeParallelIndexTaskRunner}. + * Base class for creating task that merges partial segments created by {@link PartialSegmentGenerateTask}. */ -public class PartialSegmentMergeTask extends AbstractBatchIndexTask +abstract class PartialSegmentMergeTask extends PerfectRollupWorkerTask { - public static final String TYPE = "partial_index_merge"; - private static final Logger LOG = new Logger(PartialSegmentMergeTask.class); private static final int BUFFER_SIZE = 1024 * 4; private static final int NUM_FETCH_RETRIES = 3; - private final byte[] buffer = new byte[BUFFER_SIZE]; - + private final PartialSegmentMergeIOConfig

ioConfig; private final int numAttempts; - private final PartialSegmentMergeIngestionSpec ingestionSchema; private final String supervisorTaskId; private final IndexingServiceClient indexingServiceClient; private final IndexTaskClientFactory taskClientFactory; private final HttpClient shuffleClient; + private final byte[] buffer; @JsonCreator public PartialSegmentMergeTask( // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask - @JsonProperty("id") @Nullable 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 PartialSegmentMergeIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context, - @JacksonInject IndexingServiceClient indexingServiceClient, - @JacksonInject IndexTaskClientFactory taskClientFactory, - @JacksonInject @EscalatedClient HttpClient shuffleClient + @Nullable String id, + final String groupId, + final TaskResource taskResource, + final String supervisorTaskId, + DataSchema dataSchema, + PartialSegmentMergeIOConfig

ioConfig, + ParallelIndexTuningConfig tuningConfig, + final int numAttempts, // zero-based counting + final Map context, + IndexingServiceClient indexingServiceClient, + IndexTaskClientFactory taskClientFactory, + HttpClient shuffleClient ) { super( - getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + id, groupId, taskResource, - ingestionSchema.getDataSchema().getDataSource(), + dataSchema, + tuningConfig, context ); - Preconditions.checkArgument( - ingestionSchema.getTuningConfig().isForceGuaranteedRollup(), - "forceGuaranteedRollup must be set" - ); - Preconditions.checkArgument( - ingestionSchema.getTuningConfig().getPartitionsSpec() == null - || ingestionSchema.getTuningConfig().getPartitionsSpec() instanceof HashedPartitionsSpec, - "Please use hashed_partitions for perfect rollup" - ); - Preconditions.checkArgument( - !ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals().isEmpty(), - "Missing intervals in granularitySpec" - ); - + this.ioConfig = ioConfig; this.numAttempts = numAttempts; - this.ingestionSchema = ingestionSchema; this.supervisorTaskId = supervisorTaskId; this.indexingServiceClient = indexingServiceClient; this.taskClientFactory = taskClientFactory; this.shuffleClient = shuffleClient; + this.buffer = new byte[BUFFER_SIZE]; } @JsonProperty @@ -155,57 +134,12 @@ public int getNumAttempts() return numAttempts; } - @JsonProperty("spec") - public PartialSegmentMergeIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - @JsonProperty public String getSupervisorTaskId() { return supervisorTaskId; } - @Override - public boolean requireLockExistingSegments() - { - return true; - } - - @Override - public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) - { - throw new UnsupportedOperationException( - "This method should be never called because PartialSegmentMergeTask always uses timeChunk locking" - + " but this method is supposed to be called only with segment locking." - ); - } - - @Override - public boolean isPerfectRollup() - { - return true; - } - - @Nullable - @Override - public Granularity getSegmentGranularity() - { - final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - if (granularitySpec instanceof ArbitraryGranularitySpec) { - return null; - } else { - return granularitySpec.getSegmentGranularity(); - } - } - - @Override - public String getType() - { - return TYPE; - } - @Override public boolean isReady(TaskActionClient taskActionClient) { @@ -216,8 +150,8 @@ public boolean isReady(TaskActionClient taskActionClient) public TaskStatus runTask(TaskToolbox toolbox) throws Exception { // Group partitionLocations by interval and partitionId - final Map>> intervalToPartitions = new HashMap<>(); - for (PartitionLocation location : ingestionSchema.getIOConfig().getPartitionLocations()) { + final Map>> intervalToPartitions = new HashMap<>(); + for (P location : ioConfig.getPartitionLocations()) { intervalToPartitions.computeIfAbsent(location.getInterval(), k -> new Int2ObjectOpenHashMap<>()) .computeIfAbsent(location.getPartitionId(), k -> new ArrayList<>()) .add(location); @@ -255,20 +189,18 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception new ClientBasedTaskInfoProvider(indexingServiceClient), getId(), 1, // always use a single http thread - ingestionSchema.getTuningConfig().getChatHandlerTimeout(), - ingestionSchema.getTuningConfig().getChatHandlerNumRetries() + getTuningConfig().getChatHandlerTimeout(), + getTuningConfig().getChatHandlerNumRetries() ); - final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) ingestionSchema - .getTuningConfig().getGivenOrDefaultPartitionsSpec(); - final File persistDir = toolbox.getPersistDir(); FileUtils.deleteQuietly(persistDir); FileUtils.forceMkdir(persistDir); final Set pushedSegments = mergeAndPushSegments( toolbox, - partitionsSpec, + getDataSchema(), + getTuningConfig(), persistDir, intervalToVersion, intervalToUnzippedFiles @@ -281,7 +213,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception private Map>> fetchSegmentFiles( TaskToolbox toolbox, - Map>> intervalToPartitions + Map>> intervalToPartitions ) throws IOException { final File tempDir = toolbox.getFirehoseTemporaryDir(); @@ -290,9 +222,9 @@ private Map>> fetchSegmentFiles( final Map>> intervalToUnzippedFiles = new HashMap<>(); // Fetch partition files - for (Entry>> entryPerInterval : intervalToPartitions.entrySet()) { + for (Entry>> entryPerInterval : intervalToPartitions.entrySet()) { final Interval interval = entryPerInterval.getKey(); - for (Int2ObjectMap.Entry> entryPerPartitionId : + for (Int2ObjectMap.Entry> entryPerPartitionId : entryPerInterval.getValue().int2ObjectEntrySet()) { final int partitionId = entryPerPartitionId.getIntKey(); final File partitionDir = FileUtils.getFile( @@ -302,7 +234,7 @@ private Map>> fetchSegmentFiles( Integer.toString(partitionId) ); FileUtils.forceMkdir(partitionDir); - for (PartitionLocation location : entryPerPartitionId.getValue()) { + for (P location : entryPerPartitionId.getValue()) { final File zippedFile = fetchSegmentFile(partitionDir, location); try { final File unzippedDir = new File(partitionDir, StringUtils.format("unzipped_%s", location.getSubTaskId())); @@ -324,7 +256,7 @@ private Map>> fetchSegmentFiles( } @VisibleForTesting - File fetchSegmentFile(File partitionDir, PartitionLocation location) throws IOException + File fetchSegmentFile(File partitionDir, P location) throws IOException { final File zippedFile = new File(partitionDir, StringUtils.format("temp_%s", location.getSubTaskId())); final URI uri = location.toIntermediaryDataServerURI(supervisorTaskId); @@ -348,9 +280,15 @@ File fetchSegmentFile(File partitionDir, PartitionLocation location) throws IOEx return zippedFile; } + /** + * Create a {@link ShardSpec} suitable for the desired secondary partitioning strategy. + */ + abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int partitionNum); + private Set mergeAndPushSegments( TaskToolbox toolbox, - HashedPartitionsSpec partitionsSpec, + DataSchema dataSchema, + ParallelIndexTuningConfig tuningConfig, File persistDir, Map intervalToVersion, Map>> intervalToUnzippedFiles @@ -364,15 +302,16 @@ private Set mergeAndPushSegments( final int partitionId = entryPerPartitionId.getIntKey(); final List segmentFilesToMerge = entryPerPartitionId.getValue(); final Pair> mergedFileAndDimensionNames = mergeSegmentsInSamePartition( - ingestionSchema, + dataSchema, + tuningConfig, toolbox.getIndexIO(), toolbox.getIndexMergerV9(), segmentFilesToMerge, - ingestionSchema.getTuningConfig().getMaxNumSegmentsToMerge(), + tuningConfig.getMaxNumSegmentsToMerge(), persistDir, 0 ); - final List metricNames = Arrays.stream(ingestionSchema.getDataSchema().getAggregators()) + final List metricNames = Arrays.stream(dataSchema.getAggregators()) .map(AggregatorFactory::getName) .collect(Collectors.toList()); @@ -387,12 +326,7 @@ private Set mergeAndPushSegments( null, // will be filled in the segmentPusher mergedFileAndDimensionNames.rhs, metricNames, - new HashBasedNumberedShardSpec( - partitionId, - Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), - partitionsSpec.getPartitionDimensions(), - toolbox.getObjectMapper() - ), + createShardSpec(toolbox, interval, partitionId), null, // will be filled in the segmentPusher 0 // will be filled in the segmentPusher ), @@ -408,7 +342,8 @@ private Set mergeAndPushSegments( } private static Pair> mergeSegmentsInSamePartition( - PartialSegmentMergeIngestionSpec ingestionSpec, + DataSchema dataSchema, + ParallelIndexTuningConfig tuningConfig, IndexIO indexIO, IndexMergerV9 merger, List indexes, @@ -439,11 +374,11 @@ private static Pair> mergeSegmentsInSamePartition( mergedFiles.add( merger.mergeQueryableIndex( indexesToMerge, - ingestionSpec.getDataSchema().getGranularitySpec().isRollup(), - ingestionSpec.getDataSchema().getAggregators(), + dataSchema.getGranularitySpec().isRollup(), + dataSchema.getAggregators(), outDir, - ingestionSpec.getTuningConfig().getIndexSpec(), - ingestionSpec.getTuningConfig().getSegmentWriteOutMediumFactory() + tuningConfig.getIndexSpec(), + tuningConfig.getSegmentWriteOutMediumFactory() ) ); @@ -454,7 +389,8 @@ private static Pair> mergeSegmentsInSamePartition( return Pair.of(mergedFiles.get(0), Preconditions.checkNotNull(dimensionNames, "dimensionNames")); } else { return mergeSegmentsInSamePartition( - ingestionSpec, + dataSchema, + tuningConfig, indexIO, merger, mergedFiles, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java index b7155367dd16..dc1078306ed8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java @@ -25,28 +25,29 @@ import org.joda.time.Interval; import java.net.URI; +import java.util.Objects; /** - * This class represents the intermediary data server where the partition of {@link #interval} and {@link #partitionId} - * is stored. + * This class represents the intermediary data server where the partition of {@link #interval} and + * {@link #getPartitionId()} is stored. */ -public class PartitionLocation +abstract class PartitionLocation { private final String host; private final int port; private final boolean useHttps; private final String subTaskId; private final Interval interval; - private final int partitionId; + private final T secondaryPartition; @JsonCreator public PartitionLocation( - @JsonProperty("host") String host, - @JsonProperty("port") int port, - @JsonProperty("useHttps") boolean useHttps, - @JsonProperty("subTaskId") String subTaskId, - @JsonProperty("interval") Interval interval, - @JsonProperty("partitionId") int partitionId + String host, + int port, + boolean useHttps, + String subTaskId, + Interval interval, + T secondaryPartition ) { this.host = host; @@ -54,7 +55,7 @@ public PartitionLocation( this.useHttps = useHttps; this.subTaskId = subTaskId; this.interval = interval; - this.partitionId = partitionId; + this.secondaryPartition = secondaryPartition; } @JsonProperty @@ -87,13 +88,9 @@ public Interval getInterval() return interval; } - @JsonProperty - public int getPartitionId() - { - return partitionId; - } + abstract int getPartitionId(); - URI toIntermediaryDataServerURI(String supervisorTaskId) + final URI toIntermediaryDataServerURI(String supervisorTaskId) { return URI.create( StringUtils.format( @@ -105,11 +102,35 @@ URI toIntermediaryDataServerURI(String supervisorTaskId) StringUtils.urlEncode(subTaskId), interval.getStart(), interval.getEnd(), - partitionId + getPartitionId() ) ); } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionLocation that = (PartitionLocation) o; + return port == that.port && + useHttps == that.useHttps && + Objects.equals(host, that.host) && + Objects.equals(subTaskId, that.subTaskId) && + Objects.equals(interval, that.interval) && + Objects.equals(secondaryPartition, that.secondaryPartition); + } + + @Override + public int hashCode() + { + return Objects.hash(host, port, useHttps, subTaskId, interval, secondaryPartition); + } + @Override public String toString() { @@ -119,7 +140,7 @@ public String toString() ", useHttps=" + useHttps + ", subTaskId='" + subTaskId + '\'' + ", interval=" + interval + - ", partitionId=" + partitionId + + ", secondaryPartition=" + secondaryPartition + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java index 010e84f7f8c1..79c59eda0bc8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; @@ -27,11 +26,11 @@ import java.util.Objects; /** - * Statistics about a partition created by {@link PartialSegmentGenerateTask}. Each partition is a set of data - * of the same time chunk (primary partition key) and the same partitionId (secondary partition key). This class - * holds the statistics of a single partition created by a task. + * Statistics about a partition created by {@link PartialSegmentGenerateTask}. Each partition is a + * set of data of the same time chunk (primary partition key) and the same secondary partition key + * ({@link T}). This class holds the statistics of a single partition created by a task. */ -public class PartitionStat +abstract class PartitionStat { // Host and port of the task executor private final String taskExecutorHost; @@ -40,8 +39,6 @@ public class PartitionStat // Primary partition key private final Interval interval; - // Secondary partition key - private final int partitionId; // numRows and sizeBytes are always null currently and will be filled properly in the future. @Nullable @@ -49,70 +46,65 @@ public class PartitionStat @Nullable private final Long sizeBytes; - @JsonCreator - public PartitionStat( - @JsonProperty("taskExecutorHost") String taskExecutorHost, - @JsonProperty("taskExecutorPort") int taskExecutorPort, - @JsonProperty("useHttps") boolean useHttps, - @JsonProperty("interval") Interval interval, - @JsonProperty("partitionId") int partitionId, - @JsonProperty("numRows") @Nullable Integer numRows, - @JsonProperty("sizeBytes") @Nullable Long sizeBytes + PartitionStat( + String taskExecutorHost, + int taskExecutorPort, + boolean useHttps, + Interval interval, + @Nullable Integer numRows, + @Nullable Long sizeBytes ) { this.taskExecutorHost = taskExecutorHost; this.taskExecutorPort = taskExecutorPort; this.useHttps = useHttps; this.interval = interval; - this.partitionId = partitionId; this.numRows = numRows == null ? 0 : numRows; this.sizeBytes = sizeBytes == null ? 0 : sizeBytes; } @JsonProperty - public String getTaskExecutorHost() + public final String getTaskExecutorHost() { return taskExecutorHost; } @JsonProperty - public int getTaskExecutorPort() + public final int getTaskExecutorPort() { return taskExecutorPort; } @JsonProperty - public boolean isUseHttps() + public final boolean isUseHttps() { return useHttps; } @JsonProperty - public Interval getInterval() + public final Interval getInterval() { return interval; } - @JsonProperty - public int getPartitionId() - { - return partitionId; - } - @Nullable @JsonProperty - public Integer getNumRows() + public final Integer getNumRows() { return numRows; } @Nullable @JsonProperty - public Long getSizeBytes() + public final Long getSizeBytes() { return sizeBytes; } + abstract int getPartitionId(); + + abstract T getSecondaryPartition(); + @Override public boolean equals(Object o) { @@ -125,7 +117,6 @@ public boolean equals(Object o) PartitionStat that = (PartitionStat) o; return taskExecutorPort == that.taskExecutorPort && useHttps == that.useHttps && - partitionId == that.partitionId && Objects.equals(taskExecutorHost, that.taskExecutorHost) && Objects.equals(interval, that.interval) && Objects.equals(numRows, that.numRows) && @@ -135,6 +126,6 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(taskExecutorHost, taskExecutorPort, useHttps, interval, partitionId, numRows, sizeBytes); + return Objects.hash(taskExecutorHost, taskExecutorPort, useHttps, interval, numRows, sizeBytes); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java new file mode 100644 index 000000000000..82b8211b193b --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.base.Preconditions; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + +/** + * Base class for parallel indexing perfect rollup worker tasks. + */ +abstract class PerfectRollupWorkerTask extends AbstractBatchIndexTask +{ + private final GranularitySpec granularitySpec; + private final DataSchema dataSchema; + private final ParallelIndexTuningConfig tuningConfig; + + PerfectRollupWorkerTask( + String id, + @Nullable String groupId, + @Nullable TaskResource taskResource, + DataSchema dataSchema, + ParallelIndexTuningConfig tuningConfig, + @Nullable Map context + ) + { + super(id, groupId, taskResource, dataSchema.getDataSource(), context); + + Preconditions.checkArgument( + tuningConfig.isForceGuaranteedRollup(), + "forceGuaranteedRollup must be set" + ); + + checkPartitionsSpec(tuningConfig.getGivenOrDefaultPartitionsSpec()); + + granularitySpec = dataSchema.getGranularitySpec(); + Preconditions.checkArgument( + !granularitySpec.inputIntervals().isEmpty(), + "Missing intervals in granularitySpec" + ); + + this.dataSchema = dataSchema; + this.tuningConfig = tuningConfig; + } + + private static void checkPartitionsSpec(PartitionsSpec partitionsSpec) + { + if (!partitionsSpec.isForceGuaranteedRollupCompatible()) { + String incompatibiltyMsg = partitionsSpec.getForceGuaranteedRollupIncompatiblityReason(); + String msg = "forceGuaranteedRollup is incompatible with partitionsSpec: " + incompatibiltyMsg; + throw new IllegalArgumentException(msg); + } + } + + @Override + public final int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); + } + + @Override + public final boolean requireLockExistingSegments() + { + return true; + } + + @Override + public final List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) + { + throw new UnsupportedOperationException("This task locks by timeChunk instead of segment"); + } + + @Override + public final boolean isPerfectRollup() + { + return true; + } + + @Nullable + @Override + public final Granularity getSegmentGranularity() + { + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } + } + + DataSchema getDataSchema() + { + return dataSchema; + } + + ParallelIndexTuningConfig getTuningConfig() + { + return tuningConfig; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java new file mode 100644 index 000000000000..22b621d968a9 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel.iterator; + +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowIterator; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +/** + *

+ * Build a default {@link InputRowIterator} for {@link IndexTask}s. Each {@link InputRow} is
+ * processed by the following handlers, in order:
+ *
+ *   1. Null row: If {@link InputRow} is null, invoke the null row {@link Runnable} callback.
+ *
+ *   2. Invalid timestamp: If {@link InputRow} has an invalid timestamp, throw a {@link ParseException}.
+ *
+ *   3. Absent bucket interval: If {@link InputRow} has a timestamp that does not match the
+ *      {@link GranularitySpec} bucket intervals, invoke the absent bucket interval {@link Consumer}
+ *      callback.
+ *
+ * If any of the handlers invoke their respective callback, the {@link InputRowIterator} will yield
+ * a null {@link InputRow} next; otherwise, the next {@link InputRow} is yielded.
+ * 
+ * + * @see RangePartitionIndexTaskInputRowIteratorBuilder + */ +public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRowIteratorBuilder +{ + private Firehose firehose = null; + private GranularitySpec granularitySpec = null; + private InputRowIterator.InputRowHandler nullRowHandler = null; + private InputRowIterator.InputRowHandler absentBucketIntervalHandler = null; + private List appendedInputRowHandlers = new ArrayList(); + + @Override + public DefaultIndexTaskInputRowIteratorBuilder firehose(Firehose firehose) + { + this.firehose = firehose; + return this; + } + + @Override + public DefaultIndexTaskInputRowIteratorBuilder granularitySpec(GranularitySpec granularitySpec) + { + this.granularitySpec = granularitySpec; + return this; + } + + @Override + public DefaultIndexTaskInputRowIteratorBuilder nullRowRunnable(Runnable nullRowRunnable) + { + this.nullRowHandler = inputRow -> { + if (inputRow == null) { + nullRowRunnable.run(); + return true; + } + return false; + }; + return this; + } + + @Override + public DefaultIndexTaskInputRowIteratorBuilder absentBucketIntervalConsumer( + Consumer absentBucketIntervalConsumer + ) + { + this.absentBucketIntervalHandler = inputRow -> { + Optional intervalOpt = granularitySpec.bucketInterval(inputRow.getTimestamp()); + if (!intervalOpt.isPresent()) { + absentBucketIntervalConsumer.accept(inputRow); + return true; + } + return false; + }; + return this; + } + + @Override + public InputRowIterator build() + { + Preconditions.checkNotNull(firehose, "firehose required"); + Preconditions.checkNotNull(granularitySpec, "granularitySpec required"); + Preconditions.checkNotNull(nullRowHandler, "nullRowRunnable required"); + Preconditions.checkNotNull(absentBucketIntervalHandler, "absentBucketIntervalConsumer required"); + + ImmutableList.Builder handlersBuilder = ImmutableList.builder() + .add(nullRowHandler) + .add(createInvalidTimestampHandler()) + .add(absentBucketIntervalHandler) + .addAll(appendedInputRowHandlers); + + return new InputRowIterator(firehose, handlersBuilder.build()); + } + + /** + * @param inputRowHandler Optionally, append this input row handler to the required ones. + */ + DefaultIndexTaskInputRowIteratorBuilder appendInputRowHandler(InputRowIterator.InputRowHandler inputRowHandler) + { + this.appendedInputRowHandlers.add(inputRowHandler); + return this; + } + + private InputRowIterator.InputRowHandler createInvalidTimestampHandler() + { + return inputRow -> { + if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) { + String errorMsg = StringUtils.format( + "Encountered row with timestamp that cannot be represented as a long: [%s]", + inputRow + ); + throw new ParseException(errorMsg); + } + return false; + }; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java new file mode 100644 index 000000000000..b8842e7ea9c8 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel.iterator; + +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowIterator; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; + +import java.util.function.Consumer; + +public interface IndexTaskInputRowIteratorBuilder +{ + Runnable NOOP_RUNNABLE = () -> { + }; + + Consumer NOOP_CONSUMER = inputRow -> { + }; + + /** + * @param firehose Source of {@link InputRow}s. + */ + IndexTaskInputRowIteratorBuilder firehose(Firehose firehose); + + /** + * @param granularitySpec {@link GranularitySpec} for the {@link org.apache.druid.segment.indexing.DataSchema} + * associated with the {@link Firehose}. + */ + IndexTaskInputRowIteratorBuilder granularitySpec(GranularitySpec granularitySpec); + + /** + * @param nullRowRunnable Runnable for when {@link Firehose} yields a null row. + */ + IndexTaskInputRowIteratorBuilder nullRowRunnable(Runnable nullRowRunnable); + + /** + * @param absentBucketIntervalConsumer Consumer for when {@link Firehose} yields a row with a timestamp that does not + * match the {@link GranularitySpec} bucket intervals. + */ + IndexTaskInputRowIteratorBuilder absentBucketIntervalConsumer(Consumer absentBucketIntervalConsumer); + + InputRowIterator build(); +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java index fccdec996ba2..1a87fb01f76c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java @@ -189,7 +189,7 @@ public void testSerdeTuningConfigWithDeprecatedHashedPartitionsSpec() throws IOE public void testForceGuaranteedRollupWithDynamicPartitionsSpec() { expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("HashedPartitionsSpec must be used for perfect rollup"); + expectedException.expectMessage("DynamicPartitionsSpec cannot be used for perfect rollup"); final IndexTuningConfig tuningConfig = new IndexTuningConfig( null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java new file mode 100644 index 000000000000..04ce63db4c7e --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.impl.ParseSpec; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.SegmentLoaderFactory; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanQueryConfig; +import org.apache.druid.query.scan.ScanQueryEngine; +import org.apache.druid.query.scan.ScanQueryQueryToolChest; +import org.apache.druid.query.scan.ScanQueryRunnerFactory; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.query.spec.SpecificSegmentSpec; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.loading.SegmentLoader; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +@SuppressWarnings("SameParameterValue") +abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest +{ + private static final ScanQueryRunnerFactory SCAN_QUERY_RUNNER_FACTORY = new ScanQueryRunnerFactory( + new ScanQueryQueryToolChest( + new ScanQueryConfig().setLegacy(false), + DefaultGenericQueryMetricsFactory.instance() + ), + new ScanQueryEngine(), + new ScanQueryConfig() + ); + + private final LockGranularity lockGranularity; + + AbstractMultiPhaseParallelIndexingTest(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + } + + @Before + public void setup() throws IOException + { + localDeepStorage = temporaryFolder.newFolder("localStorage"); + indexingServiceClient = new LocalIndexingServiceClient(); + initializeIntermediaryDataManager(); + } + + @After + public void teardown() + { + indexingServiceClient.shutdown(); + temporaryFolder.delete(); + } + + Set runTestTask( + ParseSpec parseSpec, + Interval interval, + FirehoseFactory firehoseFactory, + DimensionBasedPartitionsSpec partitionsSpec + ) throws Exception + { + final ParallelIndexSupervisorTask task = newTask( + parseSpec, + interval, + new ParallelIndexIOConfig(firehoseFactory, false), + partitionsSpec + ); + + actionClient = createActionClient(task); + toolbox = createTaskToolbox(task); + + prepareTaskForLocking(task); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); + Assert.assertTrue(task.isReady(actionClient)); + + TaskStatus taskStatus = task.run(toolbox); + + Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode()); + shutdownTask(task); + return actionClient.getPublishedSegments(); + } + + private ParallelIndexSupervisorTask newTask( + ParseSpec parseSpec, + Interval interval, + ParallelIndexIOConfig ioConfig, + DimensionBasedPartitionsSpec partitionsSpec + ) + { + return newTask( + parseSpec, + interval, + Granularities.DAY, + ioConfig, + new ParallelIndexTuningConfig( + null, + null, + null, + null, + null, + null, + null, + partitionsSpec, + null, + null, + null, + true, + null, + null, + null, + null, + 2, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ); + } + + private ParallelIndexSupervisorTask newTask( + ParseSpec parseSpec, + Interval interval, + Granularity segmentGranularity, + ParallelIndexIOConfig ioConfig, + ParallelIndexTuningConfig tuningConfig + ) + { + // set up ingestion spec + //noinspection unchecked + final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + getObjectMapper().convertValue( + new StringInputRowParser(parseSpec, null), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null, + getObjectMapper() + ), + ioConfig, + tuningConfig + ); + + // set up test tools + return createParallelIndexSupervisorTask( + null, + null, + ingestionSpec, + new HashMap<>(), + indexingServiceClient + ); + } + + abstract ParallelIndexSupervisorTask createParallelIndexSupervisorTask( + String id, + TaskResource taskResource, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ); + + List querySegment(DataSegment dataSegment, List columns, File tempSegmentDir) + { + Segment segment = loadSegment(dataSegment, tempSegmentDir); + final QueryRunner runner = SCAN_QUERY_RUNNER_FACTORY.createRunner(segment); + return runner.run( + QueryPlus.wrap( + new ScanQuery( + new TableDataSource("dataSource"), + new SpecificSegmentSpec( + new SegmentDescriptor( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().getPartitionNum() + ) + ), + null, + null, + 0, + 0, + null, + null, + columns, + false, + null + ) + ) + ).toList(); + } + + private Segment loadSegment(DataSegment dataSegment, File tempSegmentDir) + { + final SegmentLoader loader = new SegmentLoaderFactory(getIndexIO(), getObjectMapper()) + .manufacturate(tempSegmentDir); + try { + return loader.getSegment(dataSegment); + } + catch (SegmentLoadingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 974cfb3687db..51a8a815915a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -73,7 +73,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -109,7 +108,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase private IntermediaryDataManager intermediaryDataManager; - protected void initializeIntermeidaryDataManager() throws IOException + protected void initializeIntermediaryDataManager() throws IOException { intermediaryDataManager = new IntermediaryDataManager( new WorkerConfig(), @@ -347,32 +346,6 @@ static class TestSinglePhaseParallelIndexTaskRunner extends SinglePhaseParallelI indexingServiceClient ); } - - @Override - Iterator> subTaskSpecIterator() throws IOException - { - final Iterator> iterator = super.subTaskSpecIterator(); - return new Iterator>() - { - @Override - public boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public SubTaskSpec next() - { - try { - Thread.sleep(10); - return iterator.next(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - }; - } } static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/Factory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/Factory.java new file mode 100644 index 000000000000..3c57d02c4abe --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/Factory.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.guice.FirehoseModule; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.jackson.JacksonModule; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.easymock.EasyMock; +import org.joda.time.Duration; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +class Factory +{ + static final String AUTOMATIC_ID = null; + static final String ID = "id"; + static final String GROUP_ID = "group-id"; + static final TaskResource TASK_RESOURCE = null; + static final String SUPERVISOR_TASK_ID = "supervisor-task-id"; + static final int NUM_ATTEMPTS = 1; + static final Map CONTEXT = Collections.emptyMap(); + static final IndexingServiceClient INDEXING_SERVICE_CLIENT = null; + static final IndexTaskClientFactory TASK_CLIENT_FACTORY = null; + static final AppenderatorsManager APPENDERATORS_MANAGER = null; + static final HttpClient SHUFFLE_CLIENT = null; + static final List INPUT_INTERVALS = Collections.singletonList(Intervals.ETERNITY); + static final String TASK_EXECUTOR_HOST = "task-executor-host"; + static final int TASK_EXECUTOR_PORT = 1; + static final boolean USE_HTTPS = true; + static final Interval INTERVAL = Intervals.ETERNITY; + static final int NUM_ROWS = 2; + static final long SIZE_BYTES = 3; + static final int PARTITION_ID = 4; + static final String HOST = "host"; + static final int PORT = 1; + static final String SUBTASK_ID = "subtask-id"; + private static final ObjectMapper NESTED_OBJECT_MAPPER = TestHelper.makeJsonMapper(); + private static final String SCHEMA_TIME = "time"; + private static final String SCHEMA_DIMENSION = "dim"; + private static final String DATASOURCE = "datasource"; + + static final HashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new HashBasedNumberedShardSpec( + PARTITION_ID, + PARTITION_ID + 1, + Collections.singletonList("dim"), + Factory.NESTED_OBJECT_MAPPER + ); + + static ObjectMapper createObjectMapper() + { + InjectableValues injectableValues = new InjectableValues.Std() + .addValue(IndexingServiceClient.class, INDEXING_SERVICE_CLIENT) + .addValue(IndexTaskClientFactory.class, TASK_CLIENT_FACTORY) + .addValue(AppenderatorsManager.class, APPENDERATORS_MANAGER) + .addValue(ObjectMapper.class, NESTED_OBJECT_MAPPER) + .addValue(HttpClient.class, SHUFFLE_CLIENT); + + ObjectMapper objectMapper = new JacksonModule().jsonMapper().setInjectableValues(injectableValues); + + List firehoseModule = new FirehoseModule().getJacksonModules(); + firehoseModule.forEach(objectMapper::registerModule); + + return objectMapper; + } + + @SuppressWarnings("SameParameterValue") + static class TuningConfigBuilder + { + private PartitionsSpec partitionsSpec = + new HashedPartitionsSpec(null, 2, null); + private boolean forceGuaranteedRollup = true; + private boolean logParseExceptions = false; + private int maxParseExceptions = Integer.MAX_VALUE; + + TuningConfigBuilder partitionsSpec(PartitionsSpec partitionsSpec) + { + this.partitionsSpec = partitionsSpec; + return this; + } + + TuningConfigBuilder forceGuaranteedRollup(boolean forceGuaranteedRollup) + { + this.forceGuaranteedRollup = forceGuaranteedRollup; + return this; + } + + TuningConfigBuilder logParseExceptions(boolean logParseExceptions) + { + this.logParseExceptions = logParseExceptions; + return this; + } + + TuningConfigBuilder maxParseExceptions(int maxParseExceptions) + { + this.maxParseExceptions = maxParseExceptions; + return this; + } + + ParallelIndexTuningConfig build() + { + return new ParallelIndexTuningConfig( + 1, + null, + 3, + 4L, + 5L, + 6, + null, + partitionsSpec, + null, + null, + 10, + forceGuaranteedRollup, + false, + 14L, + null, + null, + 16, + 17, + 18L, + Duration.ZERO, + 20, + 21, + 22, + logParseExceptions, + maxParseExceptions, + 25 + ); + } + } + + static DataSchema createDataSchema(List granularitySpecInputIntervals) + { + GranularitySpec granularitySpec = new ArbitraryGranularitySpec(Granularities.DAY, granularitySpecInputIntervals); + + Map parser = NESTED_OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec(SCHEMA_TIME, "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION)), + null, + null + ), + null, + null + ), + null + ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + return new DataSchema( + DATASOURCE, + parser, + null, + granularitySpec, + null, + NESTED_OBJECT_MAPPER + ); + } + + static ParallelIndexIngestionSpec createIngestionSpec( + InlineFirehoseFactory inlineFirehoseFactory, + ParallelIndexTuningConfig tuningConfig, + DataSchema dataSchema + ) + { + ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(inlineFirehoseFactory, false); + + return new ParallelIndexIngestionSpec(dataSchema, ioConfig, tuningConfig); + } + + static class SingleDimensionPartitionsSpecBuilder + { + @Nullable + private String partitionDimension = SCHEMA_DIMENSION; + private boolean assumeGrouped = false; + + SingleDimensionPartitionsSpecBuilder partitionDimension(@Nullable String partitionDimension) + { + this.partitionDimension = partitionDimension; + return this; + } + + SingleDimensionPartitionsSpecBuilder assumeGrouped(boolean assumeGrouped) + { + this.assumeGrouped = assumeGrouped; + return this; + } + + SingleDimensionPartitionsSpec build() + { + return new SingleDimensionPartitionsSpec( + 1, + null, + partitionDimension, + assumeGrouped + ); + } + } + + static IndexTaskClientFactory createTaskClientFactory() + { + return (taskInfoProvider, callerId, numThreads, httpTimeout, numRetries) -> createTaskClient(); + } + + private static ParallelIndexSupervisorTaskClient createTaskClient() + { + ParallelIndexSupervisorTaskClient taskClient = EasyMock.niceMock(ParallelIndexSupervisorTaskClient.class); + EasyMock.replay(taskClient); + return taskClient; + } + + static String createRow(long timestamp, Object dimensionValue) + { + try { + return NESTED_OBJECT_MAPPER.writeValueAsString(ImmutableMap.of( + SCHEMA_TIME, timestamp, + SCHEMA_DIMENSION, dimensionValue + )); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReportTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReportTest.java new file mode 100644 index 000000000000..67132f39280f --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReportTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.segment.TestHelper; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +public class GeneratedPartitionsReportTest +{ + private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + + private GeneratedPartitionsReport target; + + @Before + public void setup() + { + target = new GeneratedPartitionsReport<>( + "task-id", + Collections.singletonList( + new HashPartitionStat( + Factory.TASK_EXECUTOR_HOST, + Factory.TASK_EXECUTOR_PORT, + Factory.USE_HTTPS, + Factory.INTERVAL, + Factory.PARTITION_ID, + Factory.NUM_ROWS, + Factory.SIZE_BYTES + ) + ) + ); + } + + @Test + public void serializesDeserializes() + { + TestHelper.testSerializesDeserializes( + OBJECT_MAPPER, + target, + new TypeReference>() + { + } + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java new file mode 100644 index 000000000000..248a7e3d5ded --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -0,0 +1,334 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.impl.CSVParseSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.ParseSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.scan.ScanResultValue; +import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +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.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +@RunWith(Parameterized.class) +public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPhaseParallelIndexingTest +{ + private static final ParseSpec PARSE_SPEC = new CSVParseSpec( + new TimestampSpec( + "ts", + "auto", + null + ), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")), + new ArrayList<>(), + new ArrayList<>() + ), + null, + Arrays.asList("ts", "dim1", "dim2", "val"), + false, + 0 + ); + + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK}, + new Object[]{LockGranularity.SEGMENT} + ); + } + + private File inputDir; + + public HashPartitionMultiPhaseParallelIndexingTest(LockGranularity lockGranularity) + { + super(lockGranularity); + } + + @Override + @Before + public void setup() throws IOException + { + super.setup(); + + inputDir = temporaryFolder.newFolder("data"); + // set up data + for (int i = 0; i < 10; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + for (int j = 0; j < 10; j++) { + writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", j + 1, i + 10, i)); + writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", j + 2, i + 11, 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,%d th test file\n", i + 1, i + 10, i)); + } + } + } + + @Test + public void testRun() throws Exception + { + final Set publishedSegments = runTestTask( + PARSE_SPEC, + Intervals.of("2017/2018"), + new LocalFirehoseFactory(inputDir, "test_*", null), + new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")) + ); + assertHashedPartition(publishedSegments); + } + + private void assertHashedPartition(Set publishedSegments) throws IOException + { + final Map> intervalToSegments = new HashMap<>(); + publishedSegments.forEach( + segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + final File tempSegmentDir = temporaryFolder.newFolder(); + for (List segmentsInInterval : intervalToSegments.values()) { + Assert.assertEquals(2, segmentsInInterval.size()); + for (DataSegment segment : segmentsInInterval) { + List results = querySegment(segment, ImmutableList.of("dim1", "dim2"), tempSegmentDir); + final int hash = HashBasedNumberedShardSpec.hash(getObjectMapper(), (List) results.get(0).getEvents()); + for (ScanResultValue value : results) { + Assert.assertEquals( + hash, + HashBasedNumberedShardSpec.hash(getObjectMapper(), (List) value.getEvents()) + ); + } + } + } + } + + @Override + ParallelIndexSupervisorTask createParallelIndexSupervisorTask( + String id, + TaskResource taskResource, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + return new TestSupervisorTask(id, taskResource, ingestionSchema, context, indexingServiceClient); + } + + private static class TestSupervisorTask extends TestParallelIndexSupervisorTask + { + TestSupervisorTask( + String id, + TaskResource taskResource, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + super(id, taskResource, ingestionSchema, context, indexingServiceClient); + } + + @Override + public PartialHashSegmentGenerateParallelIndexTaskRunner createPartialHashSegmentGenerateRunner(TaskToolbox toolbox) + { + return new TestPartialHashSegmentGenerateRunner(toolbox, this, getIndexingServiceClient()); + } + + @Override + public PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner( + TaskToolbox toolbox, + List> ioConfigs + ) + { + return new TestPartialHashSegmentMergeParallelIndexTaskRunner( + toolbox, + this, + ioConfigs, + getIndexingServiceClient() + ); + } + } + + private static class TestPartialHashSegmentGenerateRunner extends PartialHashSegmentGenerateParallelIndexTaskRunner + { + private TestPartialHashSegmentGenerateRunner( + TaskToolbox toolbox, + ParallelIndexSupervisorTask supervisorTask, + IndexingServiceClient indexingServiceClient + ) + { + super( + toolbox, + supervisorTask.getId(), + supervisorTask.getGroupId(), + supervisorTask.getIngestionSchema(), + supervisorTask.getContext(), + indexingServiceClient, + new LocalParallelIndexTaskClientFactory(supervisorTask), + new TestAppenderatorsManager() + ); + } + } + + private static class TestPartialHashSegmentMergeParallelIndexTaskRunner + extends PartialHashSegmentMergeParallelIndexTaskRunner + { + private final ParallelIndexSupervisorTask supervisorTask; + + private TestPartialHashSegmentMergeParallelIndexTaskRunner( + TaskToolbox toolbox, + ParallelIndexSupervisorTask supervisorTask, + List> mergeIOConfigs, + IndexingServiceClient indexingServiceClient + ) + { + super( + toolbox, + supervisorTask.getId(), + supervisorTask.getGroupId(), + supervisorTask.getIngestionSchema().getDataSchema(), + mergeIOConfigs, + supervisorTask.getIngestionSchema().getTuningConfig(), + supervisorTask.getContext(), + indexingServiceClient + ); + this.supervisorTask = supervisorTask; + } + + @Override + SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) + { + final PartialSegmentMergeIngestionSpec ingestionSpec = + new PartialSegmentMergeIngestionSpec<>( + supervisorTask.getIngestionSchema().getDataSchema(), + ioConfig, + getTuningConfig() + ); + return new SubTaskSpec( + getTaskId() + "_" + getAndIncrementNextSpecId(), + getGroupId(), + getTaskId(), + getContext(), + new InputSplit<>(ioConfig.getPartitionLocations()) + ) + { + @Override + public PartialHashSegmentMergeTask newSubTask(int numAttempts) + { + return new TestPartialHashSegmentMergeTask( + null, + getGroupId(), + null, + getSupervisorTaskId(), + numAttempts, + ingestionSpec, + getContext(), + getIndexingServiceClient(), + new LocalParallelIndexTaskClientFactory(supervisorTask), + getToolbox() + ); + } + }; + } + } + + private static class TestPartialHashSegmentMergeTask extends PartialHashSegmentMergeTask + { + private final TaskToolbox toolbox; + + private TestPartialHashSegmentMergeTask( + @Nullable String id, + String groupId, + TaskResource taskResource, + String supervisorTaskId, + int numAttempts, + PartialSegmentMergeIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient, + IndexTaskClientFactory taskClientFactory, + TaskToolbox toolbox + ) + { + super( + id, + groupId, + taskResource, + supervisorTaskId, + numAttempts, + ingestionSchema, + context, + indexingServiceClient, + taskClientFactory, + null + ); + this.toolbox = toolbox; + } + + @Override + File fetchSegmentFile(File partitionDir, HashPartitionLocation location) + { + final File zippedFile = toolbox.getIntermediaryDataManager().findPartitionFile( + getSupervisorTaskId(), + location.getSubTaskId(), + location.getInterval(), + location.getPartitionId() + ); + if (zippedFile == null) { + throw new ISE("Can't find segment file for location[%s] at path[%s]", location); + } + return zippedFile; + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java new file mode 100644 index 000000000000..9a5caf2a8968 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class HashPartitionStatTest +{ + private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + + private HashPartitionStat target; + + @Before + public void setup() + { + target = new HashPartitionStat( + Factory.TASK_EXECUTOR_HOST, + Factory.TASK_EXECUTOR_PORT, + Factory.USE_HTTPS, + Factory.INTERVAL, + Factory.PARTITION_ID, + Factory.NUM_ROWS, + Factory.SIZE_BYTES + ); + } + + @Test + public void serializesDeserializes() + { + TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); + } + + @Test + public void hasPartitionIdThatMatchesSecondaryPartition() + { + Assert.assertEquals(target.getSecondaryPartition().intValue(), target.getPartitionId()); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java deleted file mode 100644 index 6501479e0f5b..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java +++ /dev/null @@ -1,583 +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 org.apache.druid.indexing.common.task.batch.parallel; - -import com.google.common.collect.ImmutableList; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.indexing.common.SegmentLoaderFactory; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.common.task.TestAppenderatorsManager; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.query.DefaultGenericQueryMetricsFactory; -import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.query.scan.ScanQueryConfig; -import org.apache.druid.query.scan.ScanQueryEngine; -import org.apache.druid.query.scan.ScanQueryQueryToolChest; -import org.apache.druid.query.scan.ScanQueryRunnerFactory; -import org.apache.druid.query.scan.ScanResultValue; -import org.apache.druid.query.spec.SpecificSegmentSpec; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.SegmentLoader; -import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -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.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -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.ArrayList; -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.Set; - -@RunWith(Parameterized.class) -public class MultiPhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest -{ - @Parameterized.Parameters(name = "{0}") - public static Iterable constructorFeeder() - { - return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK}, - new Object[]{LockGranularity.SEGMENT} - ); - } - - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - private final LockGranularity lockGranularity; - private File inputDir; - - public MultiPhaseParallelIndexingTest(LockGranularity lockGranularity) - { - this.lockGranularity = lockGranularity; - } - - @Before - public void setup() throws IOException - { - inputDir = temporaryFolder.newFolder("data"); - // set up data - for (int i = 0; i < 10; i++) { - try (final Writer writer = - Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { - for (int j = 0; j < 10; j++) { - writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", j + 1, i + 10, i)); - writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", j + 2, i + 11, 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,%d th test file\n", i + 1, i + 10, i)); - } - } - - indexingServiceClient = new LocalIndexingServiceClient(); - localDeepStorage = temporaryFolder.newFolder("localStorage"); - initializeIntermeidaryDataManager(); - } - - @After - public void teardown() - { - indexingServiceClient.shutdown(); - temporaryFolder.delete(); - } - - @Test - public void testRun() throws Exception - { - final Set publishedSegments = runTestTask( - Intervals.of("2017/2018"), - new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")) - ); - assertHashedPartition(publishedSegments); - } - - private Set runTestTask(Interval interval, HashedPartitionsSpec partitionsSpec) throws Exception - { - final ParallelIndexSupervisorTask task = newTask( - interval, - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null), - false - ), - partitionsSpec - ); - actionClient = createActionClient(task); - toolbox = createTaskToolbox(task); - - prepareTaskForLocking(task); - task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); - Assert.assertTrue(task.isReady(actionClient)); - Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); - shutdownTask(task); - return actionClient.getPublishedSegments(); - } - - private ParallelIndexSupervisorTask newTask( - Interval interval, - ParallelIndexIOConfig ioConfig, - HashedPartitionsSpec partitionsSpec - ) - { - return newTask( - interval, - Granularities.DAY, - ioConfig, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - partitionsSpec, - null, - null, - null, - true, - null, - null, - null, - null, - 2, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) - ); - } - - private ParallelIndexSupervisorTask newTask( - Interval interval, - Granularity segmentGranularity, - ParallelIndexIOConfig ioConfig, - ParallelIndexTuningConfig tuningConfig - ) - { - // set up ingestion spec - final ParseSpec parseSpec = new CSVParseSpec( - new TimestampSpec( - "ts", - "auto", - null - ), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")), - new ArrayList<>(), - new ArrayList<>() - ), - null, - Arrays.asList("ts", "dim1", "dim2", "val"), - false, - 0 - ); - - //noinspection unchecked - final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser( - parseSpec, - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null, - getObjectMapper() - ), - ioConfig, - tuningConfig - ); - - // set up test tools - return new TestSupervisorTask( - null, - null, - ingestionSpec, - new HashMap<>(), - indexingServiceClient - ); - } - - private void assertHashedPartition(Set publishedSegments) throws IOException, SegmentLoadingException - { - final Map> intervalToSegments = new HashMap<>(); - publishedSegments.forEach( - segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) - ); - final File tempSegmentDir = temporaryFolder.newFolder(); - for (List segmentsInInterval : intervalToSegments.values()) { - Assert.assertEquals(2, segmentsInInterval.size()); - for (DataSegment segment : segmentsInInterval) { - final SegmentLoader loader = new SegmentLoaderFactory(getIndexIO(), getObjectMapper()) - .manufacturate(tempSegmentDir); - ScanQueryRunnerFactory factory = new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig().setLegacy(false), - DefaultGenericQueryMetricsFactory.instance() - ), - new ScanQueryEngine(), - new ScanQueryConfig() - ); - final QueryRunner runner = factory.createRunner(loader.getSegment(segment)); - final List results = runner.run( - QueryPlus.wrap( - new ScanQuery( - new TableDataSource("dataSource"), - new SpecificSegmentSpec( - new SegmentDescriptor( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() - ) - ), - null, - null, - 0, - 0, - null, - null, - ImmutableList.of("dim1", "dim2"), - false, - null - ) - ) - ).toList(); - final int hash = HashBasedNumberedShardSpec.hash(getObjectMapper(), (List) results.get(0).getEvents()); - for (ScanResultValue value : results) { - Assert.assertEquals( - hash, - HashBasedNumberedShardSpec.hash(getObjectMapper(), (List) value.getEvents()) - ); - } - } - } - } - - private static class TestSupervisorTask extends TestParallelIndexSupervisorTask - { - TestSupervisorTask( - String id, - TaskResource taskResource, - ParallelIndexIngestionSpec ingestionSchema, - Map context, - IndexingServiceClient indexingServiceClient - ) - { - super(id, taskResource, ingestionSchema, context, indexingServiceClient); - } - - @Override - public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerateRunner(TaskToolbox toolbox) - { - return new TestPartialSegmentGenerateRunner(toolbox, this, getIndexingServiceClient()); - } - - @Override - public PartialSegmentMergeParallelIndexTaskRunner createPartialSegmentMergeRunner( - TaskToolbox toolbox, - List ioConfigs - ) - { - return new TestPartialSegmentMergeParallelIndexTaskRunner(toolbox, this, ioConfigs, getIndexingServiceClient()); - } - } - - private static class TestPartialSegmentGenerateRunner extends PartialSegmentGenerateParallelIndexTaskRunner - { - private final ParallelIndexSupervisorTask supervisorTask; - - private TestPartialSegmentGenerateRunner( - TaskToolbox toolbox, - ParallelIndexSupervisorTask supervisorTask, - IndexingServiceClient indexingServiceClient - ) - { - super( - toolbox, - supervisorTask.getId(), - supervisorTask.getGroupId(), - supervisorTask.getIngestionSchema(), - supervisorTask.getContext(), - indexingServiceClient - ); - this.supervisorTask = supervisorTask; - } - - @Override - Iterator> subTaskSpecIterator() throws IOException - { - final Iterator> iterator = super.subTaskSpecIterator(); - return new Iterator>() - { - @Override - public boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public SubTaskSpec next() - { - try { - Thread.sleep(10); - return iterator.next(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - }; - } - - @Override - SubTaskSpec newTaskSpec(InputSplit split) - { - final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( - getIngestionSchema().getDataSchema(), - new ParallelIndexIOConfig( - getBaseFirehoseFactory().withSplit(split), - getIngestionSchema().getIOConfig().isAppendToExisting() - ), - getIngestionSchema().getTuningConfig() - ); - return new SubTaskSpec( - getTaskId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), - getTaskId(), - getContext(), - split - ) - { - @Override - public PartialSegmentGenerateTask newSubTask(int numAttempts) - { - return new PartialSegmentGenerateTask( - null, - getGroupId(), - null, - getSupervisorTaskId(), - numAttempts, - subTaskIngestionSpec, - getContext(), - getIndexingServiceClient(), - new LocalParallelIndexTaskClientFactory(supervisorTask), - new TestAppenderatorsManager() - ); - } - }; - } - } - - private static class TestPartialSegmentMergeParallelIndexTaskRunner extends PartialSegmentMergeParallelIndexTaskRunner - { - private final ParallelIndexSupervisorTask supervisorTask; - - private TestPartialSegmentMergeParallelIndexTaskRunner( - TaskToolbox toolbox, - ParallelIndexSupervisorTask supervisorTask, - List mergeIOConfigs, - IndexingServiceClient indexingServiceClient - ) - { - super( - toolbox, - supervisorTask.getId(), - supervisorTask.getGroupId(), - supervisorTask.getIngestionSchema().getDataSchema(), - mergeIOConfigs, - supervisorTask.getIngestionSchema().getTuningConfig(), - supervisorTask.getContext(), - indexingServiceClient - ); - this.supervisorTask = supervisorTask; - } - - @Override - Iterator> subTaskSpecIterator() - { - final Iterator> iterator = super.subTaskSpecIterator(); - return new Iterator>() - { - @Override - public boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public SubTaskSpec next() - { - try { - Thread.sleep(10); - return iterator.next(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - }; - } - - @Override - SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) - { - final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec( - supervisorTask.getIngestionSchema().getDataSchema(), - ioConfig, - getTuningConfig() - ); - return new SubTaskSpec( - getTaskId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), - getTaskId(), - getContext(), - new InputSplit<>(ioConfig.getPartitionLocations()) - ) - { - @Override - public PartialSegmentMergeTask newSubTask(int numAttempts) - { - return new TestPartialSegmentMergeTask( - null, - getGroupId(), - null, - getSupervisorTaskId(), - numAttempts, - ingestionSpec, - getContext(), - getIndexingServiceClient(), - new LocalParallelIndexTaskClientFactory(supervisorTask), - getToolbox() - ); - } - }; - } - } - - private static class TestPartialSegmentMergeTask extends PartialSegmentMergeTask - { - private final TaskToolbox toolbox; - - private TestPartialSegmentMergeTask( - @Nullable String id, - String groupId, - TaskResource taskResource, - String supervisorTaskId, - int numAttempts, - PartialSegmentMergeIngestionSpec ingestionSchema, - Map context, - IndexingServiceClient indexingServiceClient, - IndexTaskClientFactory taskClientFactory, - TaskToolbox toolboxo - ) - { - super( - id, - groupId, - taskResource, - supervisorTaskId, - numAttempts, - ingestionSchema, - context, - indexingServiceClient, - taskClientFactory, - null - ); - this.toolbox = toolboxo; - } - - @Override - File fetchSegmentFile(File partitionDir, PartitionLocation location) - { - final File zippedFile = toolbox.getIntermediaryDataManager().findPartitionFile( - getSupervisorTaskId(), - location.getSubTaskId(), - location.getInterval(), - location.getPartitionId() - ); - if (zippedFile == null) { - throw new ISE("Can't find segment file for location[%s] at path[%s]", location); - } - return zippedFile; - } - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index c8857aaf105b..6d77f0fc2691 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -31,6 +31,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -48,6 +49,7 @@ import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; +import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; @@ -66,6 +68,7 @@ public class ParallelIndexSupervisorTaskSerdeTest { private static final ObjectMapper OBJECT_MAPPER = createObjectMapper(); + private static final List INTERVALS = Collections.singletonList(Intervals.of("2018/2019")); private static ObjectMapper createObjectMapper() { @@ -86,7 +89,7 @@ public void serde() throws IOException ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTaskBuilder() .ingestionSpec( new ParallelIndexIngestionSpecBuilder() - .inputIntervals(Collections.singletonList(Intervals.of("2018/2019"))) + .inputIntervals(INTERVALS) .build() ) .build(); @@ -115,11 +118,11 @@ public void forceGuaranteedRollupWithMissingIntervals() } @Test - public void forceGuaranteedRollupWithMissingNumShards() + public void forceGuaranteedRollupWithHashPartitionsMissingNumShards() { expectedException.expect(IllegalStateException.class); expectedException.expectMessage( - "forceGuaranteedRollup is set but numShards is missing in partitionsSpec" + "forceGuaranteedRollup is incompatible with partitionsSpec: numShards must be specified" ); Integer numShards = null; @@ -128,6 +131,44 @@ public void forceGuaranteedRollupWithMissingNumShards() new ParallelIndexIngestionSpecBuilder() .forceGuaranteedRollup(true) .partitionsSpec(new HashedPartitionsSpec(null, numShards, null)) + .inputIntervals(INTERVALS) + .build() + ) + .build(); + } + + @Test + public void forceGuaranteedRollupWithHashPartitionsValid() + { + Integer numShards = 2; + ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTaskBuilder() + .ingestionSpec( + new ParallelIndexIngestionSpecBuilder() + .forceGuaranteedRollup(true) + .partitionsSpec(new HashedPartitionsSpec(null, numShards, null)) + .inputIntervals(INTERVALS) + .build() + ) + .build(); + + PartitionsSpec partitionsSpec = task.getIngestionSchema().getTuningConfig().getPartitionsSpec(); + Assert.assertThat(partitionsSpec, CoreMatchers.instanceOf(HashedPartitionsSpec.class)); + } + + @Test + public void forceGuaranteedRollupWithSingleDimPartitionsInvalid() + { + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage( + "forceGuaranteedRollup is incompatible with partitionsSpec: single_dim partitions unsupported" + ); + + new ParallelIndexSupervisorTaskBuilder() + .ingestionSpec( + new ParallelIndexIngestionSpecBuilder() + .forceGuaranteedRollup(true) + .partitionsSpec(new SingleDimensionPartitionsSpec(1, null, "a", true)) + .inputIntervals(INTERVALS) .build() ) .build(); @@ -207,6 +248,7 @@ private static class ParallelIndexIngestionSpecBuilder @Nullable PartitionsSpec partitionsSpec = null; + @SuppressWarnings("SameParameterValue") ParallelIndexIngestionSpecBuilder inputIntervals(List inputIntervals) { this.inputIntervals = inputIntervals; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java new file mode 100644 index 000000000000..92206215d92f --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class PartialHashSegmentGenerateTaskTest +{ + private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final ParallelIndexIngestionSpec INGESTION_SPEC = Factory.createIngestionSpec( + new InlineFirehoseFactory("data"), + new Factory.TuningConfigBuilder().build(), + Factory.createDataSchema(Factory.INPUT_INTERVALS) + ); + + private PartialSegmentGenerateTask target; + + @Before + public void setup() + { + target = new PartialHashSegmentGenerateTask( + Factory.AUTOMATIC_ID, + Factory.GROUP_ID, + Factory.TASK_RESOURCE, + Factory.SUPERVISOR_TASK_ID, + Factory.NUM_ATTEMPTS, + INGESTION_SPEC, + Factory.CONTEXT, + Factory.INDEXING_SERVICE_CLIENT, + Factory.TASK_CLIENT_FACTORY, + Factory.APPENDERATORS_MANAGER + ); + } + + @Test + public void serializesDeserializes() + { + TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); + } + + @Test + public void hasCorrectPrefixForAutomaticId() + { + String id = target.getId(); + Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentGenerateTask.TYPE)); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java new file mode 100644 index 000000000000..669a18b8c079 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.segment.TestHelper; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +public class PartialHashSegmentMergeTaskTest +{ + private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( + Factory.HOST, + Factory.PORT, + Factory.USE_HTTPS, + Factory.SUBTASK_ID, + Factory.INTERVAL, + Factory.PARTITION_ID + ); + private static final PartialSegmentMergeIOConfig IO_CONFIG = + new PartialSegmentMergeIOConfig<>(Collections.singletonList(HASH_PARTITION_LOCATION)); + private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( + null, + 1, + Collections.emptyList() + ); + private static final PartialSegmentMergeIngestionSpec INGESTION_SPEC = + new PartialSegmentMergeIngestionSpec<>( + Factory.createDataSchema(Factory.INPUT_INTERVALS), + IO_CONFIG, + new Factory.TuningConfigBuilder() + .partitionsSpec(PARTITIONS_SPEC) + .build() + ); + + private PartialHashSegmentMergeTask target; + + @Before + public void setup() + { + target = new PartialHashSegmentMergeTask( + Factory.AUTOMATIC_ID, + Factory.GROUP_ID, + Factory.TASK_RESOURCE, + Factory.SUPERVISOR_TASK_ID, + Factory.NUM_ATTEMPTS, + INGESTION_SPEC, + Factory.CONTEXT, + Factory.INDEXING_SERVICE_CLIENT, + Factory.TASK_CLIENT_FACTORY, + Factory.SHUFFLE_CLIENT + ); + } + + @Test + public void serializesDeserializes() + { + TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); + } + + @Test + public void hasCorrectPrefixForAutomaticId() + { + String id = target.getId(); + Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentMergeTask.TYPE)); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java new file mode 100644 index 000000000000..4e041d665acb --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTaskTest.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class PerfectRollupWorkerTaskTest +{ + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void requiresForceGuaranteedRollup() + { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("forceGuaranteedRollup must be set"); + + new PerfectRollupWorkerTaskBuilder() + .forceGuaranteedRollup(false) + .build(); + } + + @Test + public void failsWithInvalidPartitionsSpec() + { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("forceGuaranteedRollup is incompatible with partitionsSpec"); + + new PerfectRollupWorkerTaskBuilder() + .partitionsSpec(HashedPartitionsSpec.defaultSpec()) + .build(); + } + + @Test + public void requiresGranularitySpecInputIntervals() + { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Missing intervals in granularitySpec"); + + new PerfectRollupWorkerTaskBuilder() + .granularitySpecInputIntervals(Collections.emptyList()) + .build(); + } + + @Test + public void succeedsWithValidPartitionsSpec() + { + new PerfectRollupWorkerTaskBuilder().build(); + } + + @SuppressWarnings("SameParameterValue") + private static class PerfectRollupWorkerTaskBuilder + { + private static final PartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( + null, + 1, + null, + null, + null + ); + + private List granularitySpecInputIntervals = Collections.singletonList(Intervals.ETERNITY); + private boolean forceGuaranteedRollup = true; + private PartitionsSpec partitionsSpec = PARTITIONS_SPEC; + + PerfectRollupWorkerTaskBuilder granularitySpecInputIntervals(List granularitySpecInputIntervals) + { + this.granularitySpecInputIntervals = granularitySpecInputIntervals; + return this; + } + + PerfectRollupWorkerTaskBuilder forceGuaranteedRollup(boolean forceGuaranteedRollup) + { + this.forceGuaranteedRollup = forceGuaranteedRollup; + return this; + } + + PerfectRollupWorkerTaskBuilder partitionsSpec(PartitionsSpec partitionsSpec) + { + this.partitionsSpec = partitionsSpec; + return this; + } + + PerfectRollupWorkerTask build() + { + return new TestPerfectRollupWorkerTask( + "id", + "group-id", + null, + createDataSchema(granularitySpecInputIntervals), + createTuningConfig(forceGuaranteedRollup, partitionsSpec), + null + ); + } + + private static DataSchema createDataSchema(List granularitySpecInputIntervals) + { + GranularitySpec granularitySpec = EasyMock.mock(GranularitySpec.class); + EasyMock.expect(granularitySpec.inputIntervals()).andStubReturn(granularitySpecInputIntervals); + EasyMock.replay(granularitySpec); + + DataSchema dataSchema = EasyMock.mock(DataSchema.class); + EasyMock.expect(dataSchema.getDataSource()).andStubReturn("datasource"); + EasyMock.expect(dataSchema.getGranularitySpec()).andStubReturn(granularitySpec); + EasyMock.replay(dataSchema); + return dataSchema; + } + + private static ParallelIndexTuningConfig createTuningConfig( + boolean forceGuaranteedRollup, + PartitionsSpec partitionsSpec + ) + { + ParallelIndexTuningConfig tuningConfig = EasyMock.mock(ParallelIndexTuningConfig.class); + EasyMock.expect(tuningConfig.isForceGuaranteedRollup()).andStubReturn(forceGuaranteedRollup); + EasyMock.expect(tuningConfig.getGivenOrDefaultPartitionsSpec()).andStubReturn(partitionsSpec); + EasyMock.replay(tuningConfig); + return tuningConfig; + } + } + + private static class TestPerfectRollupWorkerTask extends PerfectRollupWorkerTask + { + TestPerfectRollupWorkerTask( + String id, + @Nullable String groupId, + @Nullable TaskResource taskResource, + DataSchema dataSchema, + ParallelIndexTuningConfig tuningConfig, + @Nullable Map context + ) + { + super(id, groupId, taskResource, dataSchema, tuningConfig, context); + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) + { + throw new UnsupportedOperationException(); + } + + @Override + public String getType() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) + { + throw new UnsupportedOperationException(); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java new file mode 100644 index 000000000000..6fe1428809ff --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel.iterator; + +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowIterator; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; + +import java.util.Collections; +import java.util.List; +import java.util.function.Consumer; + +@RunWith(Enclosed.class) +public class DefaultIndexTaskInputRowIteratorBuilderTest +{ + public static class BuildTest + { + private static final Firehose FIRHOSE = EasyMock.mock(Firehose.class); + private static final GranularitySpec GRANULARITY_SPEC = EasyMock.mock(GranularitySpec.class); + private static final Runnable NULL_ROW_RUNNABLE = IndexTaskInputRowIteratorBuilder.NOOP_RUNNABLE; + private static final Consumer ABSENT_BUCKET_INTERVAL_CONSUMER = + IndexTaskInputRowIteratorBuilder.NOOP_CONSUMER; + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void requiresFirehose() + { + exception.expect(NullPointerException.class); + exception.expectMessage("firehose required"); + + new DefaultIndexTaskInputRowIteratorBuilder() + .granularitySpec(GRANULARITY_SPEC) + .nullRowRunnable(NULL_ROW_RUNNABLE) + .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) + .build(); + } + + @Test + public void requiresGranularitySpec() + { + exception.expect(NullPointerException.class); + exception.expectMessage("granularitySpec required"); + + new DefaultIndexTaskInputRowIteratorBuilder() + .firehose(FIRHOSE) + .nullRowRunnable(NULL_ROW_RUNNABLE) + .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) + .build(); + } + + @Test + public void requiresNullRowHandler() + { + exception.expect(NullPointerException.class); + exception.expectMessage("nullRowRunnable required"); + + new DefaultIndexTaskInputRowIteratorBuilder() + .firehose(FIRHOSE) + .granularitySpec(GRANULARITY_SPEC) + .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) + .build(); + } + + @Test + public void requiresAbsentBucketIntervalHandler() + { + exception.expect(NullPointerException.class); + exception.expectMessage("absentBucketIntervalConsumer required"); + + new DefaultIndexTaskInputRowIteratorBuilder() + .firehose(FIRHOSE) + .granularitySpec(GRANULARITY_SPEC) + .nullRowRunnable(NULL_ROW_RUNNABLE) + .build(); + } + + @Test + public void succeedsIfAllRequiredPresent() + { + new DefaultIndexTaskInputRowIteratorBuilder() + .firehose(FIRHOSE) + .granularitySpec(GRANULARITY_SPEC) + .nullRowRunnable(NULL_ROW_RUNNABLE) + .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) + .build(); + } + } + + public static class HandlerTest + { + private static final Factory.HandlerTester HANDLER_TESTER = + Factory.createHandlerTester(DefaultIndexTaskInputRowIteratorBuilder::new); + private static final InputRow NO_NEXT_INPUT_ROW = null; + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void invokesNullRowHandlerFirst() + { + DateTime invalidTimestamp = new DateTime(Long.MAX_VALUE); + Firehose nullRowFirehose = Factory.createFirehose(null); + GranularitySpec absentBucketIntervalGranularitySpec = + Factory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); + + List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( + nullRowFirehose, + absentBucketIntervalGranularitySpec, + NO_NEXT_INPUT_ROW + ); + + Assert.assertEquals(Collections.singletonList(Factory.HandlerTester.Handler.NULL_ROW), handlerInvocationHistory); + } + + @Test + public void invokesInvalidTimestampHandlerBeforeAbsentBucketIntervalHandler() + { + DateTime invalidTimestamp = new DateTime(Long.MAX_VALUE); + InputRow inputRow = Factory.createInputRow(invalidTimestamp); + Firehose firehose = Factory.createFirehose(inputRow); + GranularitySpec absentBucketIntervalGranularitySpec = + Factory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); + + exception.expect(ParseException.class); + exception.expectMessage("Encountered row with timestamp that cannot be represented as a long"); + + HANDLER_TESTER.invokeHandlers(firehose, absentBucketIntervalGranularitySpec, NO_NEXT_INPUT_ROW); + } + + @Test + public void invokesAbsentBucketIntervalHandlerLast() + { + DateTime timestamp = Factory.TIMESTAMP; + InputRow inputRow = Factory.createInputRow(timestamp); + Firehose firehose = Factory.createFirehose(inputRow); + GranularitySpec absentBucketIntervalGranularitySpec = Factory.createAbsentBucketIntervalGranularitySpec(timestamp); + + List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( + firehose, + absentBucketIntervalGranularitySpec, + NO_NEXT_INPUT_ROW + ); + + Assert.assertEquals( + Collections.singletonList(Factory.HandlerTester.Handler.ABSENT_BUCKET_INTERVAL), + handlerInvocationHistory + ); + } + + @Test + public void invokesAppendedHandlersLast() + { + DateTime timestamp = Factory.TIMESTAMP; + InputRow inputRow = Factory.createInputRow(timestamp); + Firehose firehose = Factory.createFirehose(inputRow); + GranularitySpec granularitySpec = Factory.createGranularitySpec(timestamp, Factory.PRESENT_BUCKET_INTERVAL_OPT); + + List appendedHandlers = Collections.singletonList(row -> true); + + List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( + firehose, + granularitySpec, + appendedHandlers, + NO_NEXT_INPUT_ROW + ); + + Assert.assertEquals( + Collections.singletonList(Factory.HandlerTester.Handler.APPENDED), + handlerInvocationHistory + ); + } + + @Test + public void doesNotInvokeHandlersIfRowValid() + { + DateTime timestamp = new DateTime(0); + InputRow inputRow = Factory.createInputRow(timestamp); + Firehose firehose = Factory.createFirehose(inputRow); + GranularitySpec granularitySpec = Factory.createGranularitySpec(timestamp, Factory.PRESENT_BUCKET_INTERVAL_OPT); + + List handlerInvocationHistory = + HANDLER_TESTER.invokeHandlers(firehose, granularitySpec, inputRow); + + Assert.assertEquals(Collections.emptyList(), handlerInvocationHistory); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java new file mode 100644 index 000000000000..f9efcc6dc3ac --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel.iterator; + +import com.google.common.base.Optional; +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowIterator; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +class Factory +{ + static final DateTime TIMESTAMP = new DateTime(0); + static final String DIMENSION = "dimension"; + static final Optional PRESENT_BUCKET_INTERVAL_OPT = Optional.of(Intervals.ETERNITY); + + static InputRow createInputRow(DateTime timestamp) + { + return createInputRow(timestamp, Collections.singletonList(DIMENSION)); + } + + static InputRow createInputRow(DateTime timestamp, List dimensionValues) + { + InputRow inputRow = EasyMock.mock(InputRow.class); + EasyMock.expect(inputRow.getTimestamp()).andStubReturn(timestamp); + EasyMock.expect(inputRow.getDimension(DIMENSION)).andStubReturn(dimensionValues); + EasyMock.replay(inputRow); + return inputRow; + } + + static Firehose createFirehose(InputRow inputRow) + { + Firehose firehose = EasyMock.mock(Firehose.class); + try { + EasyMock.expect(firehose.hasMore()).andStubReturn(true); + EasyMock.expect(firehose.nextRow()).andStubReturn(inputRow); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + EasyMock.replay(firehose); + + return firehose; + } + + static GranularitySpec createAbsentBucketIntervalGranularitySpec(DateTime timestamp) + { + return createGranularitySpec(timestamp, Optional.absent()); + } + + static GranularitySpec createGranularitySpec(DateTime timestamp, Optional bucketIntervalOpt) + { + GranularitySpec granularitySpec = EasyMock.mock(GranularitySpec.class); + EasyMock.expect(granularitySpec.bucketInterval(timestamp)).andStubReturn(bucketIntervalOpt); + EasyMock.replay(granularitySpec); + return granularitySpec; + } + + static HandlerTester createHandlerTester(Supplier iteratorBuilderSupplier) + { + return new HandlerTester(iteratorBuilderSupplier); + } + + static class HandlerTester + { + enum Handler + { + NULL_ROW, + ABSENT_BUCKET_INTERVAL, + APPENDED + } + + private final Supplier iteratorBuilderSupplier; + + private HandlerTester(Supplier iteratorBuilderSupplier) + { + this.iteratorBuilderSupplier = iteratorBuilderSupplier; + } + + List invokeHandlers( + Firehose firehose, + GranularitySpec granularitySpec, + InputRow expectedNextInputRow + ) + { + return invokeHandlers( + firehose, + granularitySpec, + Collections.emptyList(), + expectedNextInputRow + ); + } + + List invokeHandlers( + Firehose firehose, + GranularitySpec granularitySpec, + List appendedHandlers, + InputRow expectedNextInputRow + ) + { + List handlerInvocationHistory = new ArrayList<>(); + IndexTaskInputRowIteratorBuilder iteratorBuilder = iteratorBuilderSupplier.get() + .firehose(firehose) + .granularitySpec(granularitySpec) + .nullRowRunnable(() -> handlerInvocationHistory.add(Handler.NULL_ROW)) + .absentBucketIntervalConsumer(row -> handlerInvocationHistory.add(Handler.ABSENT_BUCKET_INTERVAL)); + + if (iteratorBuilder instanceof DefaultIndexTaskInputRowIteratorBuilder) { + appendedHandlers.stream() + .peek(handler -> handlerInvocationHistory.add(Handler.APPENDED)) + .forEach(((DefaultIndexTaskInputRowIteratorBuilder) iteratorBuilder)::appendInputRowHandler); + } + + InputRowIterator iterator = iteratorBuilder.build(); + + InputRow nextInputRow = iterator.next(); + Assert.assertEquals(expectedNextInputRow, nextInputRow); + + return handlerInvocationHistory; + } + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index c846f2747ace..2a7e0f5956f2 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -21,8 +21,8 @@ import com.google.inject.Inject; import org.apache.commons.io.IOUtils; -import org.apache.druid.indexing.common.task.batch.parallel.PartialSegmentGenerateTask; -import org.apache.druid.indexing.common.task.batch.parallel.PartialSegmentMergeTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -259,8 +259,8 @@ private long countCompleteSubTasks(final String dataSource, final boolean perfec if (!perfectRollup) { return t.getType().equals(SinglePhaseSubTask.TYPE); } else { - return t.getType().equalsIgnoreCase(PartialSegmentGenerateTask.TYPE) - || t.getType().equalsIgnoreCase(PartialSegmentMergeTask.TYPE); + return t.getType().equalsIgnoreCase(PartialHashSegmentGenerateTask.TYPE) + || t.getType().equalsIgnoreCase(PartialHashSegmentMergeTask.TYPE); } }) .count(); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITParallelIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITParallelIndexTest.java index 3b335a2a04ac..be99de14933f 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITParallelIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITParallelIndexTest.java @@ -48,18 +48,20 @@ public class ITParallelIndexTest extends AbstractITBatchIndexTest @DataProvider public static Object[][] resources() { - return new Object[][]{{false}, {true}}; + return new Object[][]{ + {new DynamicPartitionsSpec(null, null)}, + {new HashedPartitionsSpec(null, 2, null)} + }; } @Test(dataProvider = "resources") - public void testIndexData(boolean forceGuaranteedRollup) throws Exception + public void testIndexData(PartitionsSpec partitionsSpec) throws Exception { try (final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()); final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix()) ) { - final PartitionsSpec partitionsSpec = forceGuaranteedRollup - ? new HashedPartitionsSpec(null, 2, null) - : new DynamicPartitionsSpec(null, null); + boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible(); + final Function rollupTransform = spec -> { try { spec = StringUtils.replace( diff --git a/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java b/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java index 1830ef99b848..573af0a0828c 100644 --- a/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java +++ b/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java @@ -49,6 +49,7 @@ static SimpleModule register(SimpleModule module) module.addDeserializer(DateTime.class, new DateTimeDeserializer()); module.addSerializer(DateTime.class, ToStringSerializer.instance); module.addDeserializer(Interval.class, new JodaStuff.IntervalDeserializer()); + module.addKeyDeserializer(Interval.class, new JodaStuff.IntervalKeyDeserializer()); module.addSerializer(Interval.class, ToStringSerializer.instance); JsonDeserializer periodDeserializer = new PeriodDeserializer(); module.addDeserializer(Period.class, (JsonDeserializer) periodDeserializer); @@ -76,6 +77,15 @@ public Interval deserialize(JsonParser jsonParser, DeserializationContext deseri } } + private static class IntervalKeyDeserializer extends KeyDeserializer + { + @Override + public Object deserializeKey(String key, DeserializationContext ctxt) + { + return Intervals.of(key); + } + } + private static class DateTimeKeyDeserializer extends KeyDeserializer { @Override diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 53a47631354f..1d7e5faa28ec 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; @@ -39,10 +40,13 @@ import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.junit.Assert; +import java.io.IOException; +import java.io.UncheckedIOException; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -389,4 +393,61 @@ public static Map createExpectedMap(Object... vals) } return theVals; } + + public static void testSerializesDeserializes(Object object) + { + testSerializesDeserializes(JSON_MAPPER, object); + } + + public static void testSerializesDeserializes(ObjectMapper objectMapper, Object object) + { + testSerializesDeserializes( + objectMapper, + object, + serialized -> { + try { + return objectMapper.readValue(serialized, object.getClass()); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + ); + } + + public static void testSerializesDeserializes( + ObjectMapper objectMapper, + Object object, + TypeReference typeReference + ) + { + testSerializesDeserializes( + objectMapper, + object, + serialized -> { + try { + return objectMapper.readValue(serialized, typeReference); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + ); + } + + private static void testSerializesDeserializes( + ObjectMapper objectMapper, + Object object, + Function readValueFunction + ) + { + try { + String serialized = objectMapper.writeValueAsString(object); + Object deserialized = readValueFunction.apply(serialized); + Assert.assertEquals(serialized, objectMapper.writeValueAsString(deserialized)); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java index 31cb38c8a58f..7d0569fe5945 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.Firehose; @@ -41,8 +42,9 @@ public class InlineFirehoseFactory implements FiniteFirehoseFactory Date: Mon, 11 Nov 2019 11:30:48 -0800 Subject: [PATCH 02/12] Fix forbidden apis and pmd warnings --- .../batch/parallel/PartialHashSegmentGenerateTask.java | 3 +-- .../DefaultIndexTaskInputRowIteratorBuilderTest.java | 7 ++++--- .../common/task/batch/parallel/iterator/Factory.java | 3 ++- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index 17c3db15fd0d..a85f057dbb8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -31,7 +31,6 @@ import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; -import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; @@ -47,7 +46,7 @@ * The worker task of {@link PartialHashSegmentGenerateParallelIndexTaskRunner}. This task partitions input data by * hashing the segment granularity and partition dimensions in {@link * org.apache.druid.indexer.partitions.PartitionsSpec}. Partitioned segments are stored in local storage using {@link - * ShuffleDataSegmentPusher}. + * org.apache.druid.indexing.worker.ShuffleDataSegmentPusher}. */ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java index 6fe1428809ff..23fdd2a9b4ab 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java @@ -22,6 +22,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowIterator; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.easymock.EasyMock; @@ -127,7 +128,7 @@ public static class HandlerTest @Test public void invokesNullRowHandlerFirst() { - DateTime invalidTimestamp = new DateTime(Long.MAX_VALUE); + DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); Firehose nullRowFirehose = Factory.createFirehose(null); GranularitySpec absentBucketIntervalGranularitySpec = Factory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); @@ -144,7 +145,7 @@ public void invokesNullRowHandlerFirst() @Test public void invokesInvalidTimestampHandlerBeforeAbsentBucketIntervalHandler() { - DateTime invalidTimestamp = new DateTime(Long.MAX_VALUE); + DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); InputRow inputRow = Factory.createInputRow(invalidTimestamp); Firehose firehose = Factory.createFirehose(inputRow); GranularitySpec absentBucketIntervalGranularitySpec = @@ -202,7 +203,7 @@ public void invokesAppendedHandlersLast() @Test public void doesNotInvokeHandlersIfRowValid() { - DateTime timestamp = new DateTime(0); + DateTime timestamp = DateTimes.utc(0); InputRow inputRow = Factory.createInputRow(timestamp); Firehose firehose = Factory.createFirehose(inputRow); GranularitySpec granularitySpec = Factory.createGranularitySpec(timestamp, Factory.PRESENT_BUCKET_INTERVAL_OPT); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java index f9efcc6dc3ac..a9a146ba90a6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java @@ -23,6 +23,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowIterator; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.easymock.EasyMock; @@ -39,7 +40,7 @@ class Factory { - static final DateTime TIMESTAMP = new DateTime(0); + static final DateTime TIMESTAMP = DateTimes.utc(0); static final String DIMENSION = "dimension"; static final Optional PRESENT_BUCKET_INTERVAL_OPT = Optional.of(Intervals.ETERNITY); From 36b8e6559825ccdd06ca9a164258bdef0a1c0f36 Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Mon, 11 Nov 2019 11:56:14 -0800 Subject: [PATCH 03/12] Fix analyze dependencies warnings --- indexing-service/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index d7df2c07fad5..2a71a9a72c56 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -242,6 +242,11 @@ hamcrest-all test + + org.hamcrest + hamcrest-core + test + org.assertj assertj-core From 206c3aa2057719c75aa9002488f7770c9a0e54f2 Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Mon, 11 Nov 2019 15:21:41 -0800 Subject: [PATCH 04/12] Fix IndexTask json and add IT diags --- .travis.yml | 3 +++ .../java/org/apache/druid/indexing/common/task/IndexTask.java | 2 +- .../batch/parallel/PartialHashSegmentGenerateTaskTest.java | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index f4aa88b4bdcc..2ce508db7369 100644 --- a/.travis.yml +++ b/.travis.yml @@ -277,6 +277,9 @@ jobs: echo $v dmesg ======================== ; docker exec -it druid-$v sh -c 'dmesg | tail -3' ; done + - for v in ~/shared/tasklogs/*.log ; do + echo $v logtail ======================== ; tail -100 $v ; + done - &integration_kafka_index name: "kafka index integration test" diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index cadcb1cb5d73..48e207e45709 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1352,7 +1352,7 @@ public int getMaxPendingPersists() */ @Deprecated @JsonProperty - public static boolean isBuildV9Directly() + public boolean isBuildV9Directly() { return true; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java index 92206215d92f..e7f55a35c27c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java @@ -36,7 +36,7 @@ public class PartialHashSegmentGenerateTaskTest Factory.createDataSchema(Factory.INPUT_INTERVALS) ); - private PartialSegmentGenerateTask target; + private PartialHashSegmentGenerateTask target; @Before public void setup() From e09a75468f0d71113a3be2541882ded0d5679a4f Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Mon, 11 Nov 2019 17:46:54 -0800 Subject: [PATCH 05/12] Fix parallel index supervisor<->worker serde --- .../GeneratedHashPartitionsReport.java | 44 ++++++++++++ .../parallel/GeneratedPartitionsReport.java | 13 +--- .../parallel/ParallelIndexSupervisorTask.java | 41 +++++++---- ...egmentGenerateParallelIndexTaskRunner.java | 2 +- .../PartialHashSegmentGenerateTask.java | 19 ++++-- .../PartialHashSegmentMergeIOConfig.java | 40 +++++++++++ .../PartialHashSegmentMergeIngestionSpec.java | 37 ++++++++++ ...shSegmentMergeParallelIndexTaskRunner.java | 10 +-- .../parallel/PartialHashSegmentMergeTask.java | 6 +- .../parallel/PartialSegmentGenerateTask.java | 15 ++-- .../parallel/PartialSegmentMergeIOConfig.java | 8 +-- .../PartialSegmentMergeIngestionSpec.java | 15 ++-- .../task/batch/parallel/SubTaskReport.java | 2 +- ...=> GeneratedHashPartitionsReportTest.java} | 15 ++-- ...rtitionMultiPhaseParallelIndexingTest.java | 12 ++-- .../PartialHashSegmentMergeIOConfigTest.java | 54 +++++++++++++++ ...tialHashSegmentMergeIngestionSpecTest.java | 68 +++++++++++++++++++ .../PartialHashSegmentMergeTaskTest.java | 8 +-- .../org/apache/druid/segment/TestHelper.java | 44 +----------- 19 files changed, 327 insertions(+), 126 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/{GeneratedPartitionsReportTest.java => GeneratedHashPartitionsReportTest.java} (80%) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java new file mode 100644 index 000000000000..27266031d43e --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** + * Report containing the {@link HashPartitionStat}s created by a {@link PartialHashSegmentGenerateTask}. + * This report is collected by {@link ParallelIndexSupervisorTask} and + * used to generate {@link PartialHashSegmentMergeIOConfig}. + */ +public class GeneratedHashPartitionsReport extends GeneratedPartitionsReport implements SubTaskReport +{ + public static final String TYPE = "generated_partitions"; + + @JsonCreator + public GeneratedHashPartitionsReport( + @JsonProperty("taskId") String taskId, + @JsonProperty("partitionStats") List partitionStats + ) + { + super(taskId, partitionStats); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java index 9cf72ddbd6ee..23449dcefeee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; @@ -28,20 +27,14 @@ /** * Report containing the {@link PartitionStat}s created by a {@link PartialSegmentGenerateTask}. * This report is collected by {@link ParallelIndexSupervisorTask} and - * used to generate {@link PartialSegmentMergeIOConfig}. + * used to generate {@link PartialHashSegmentMergeIOConfig}. */ -public class GeneratedPartitionsReport implements SubTaskReport +abstract class GeneratedPartitionsReport implements SubTaskReport { - public static final String TYPE = "generated_partitions"; - private final String taskId; private final List partitionStats; - @JsonCreator - public GeneratedPartitionsReport( - @JsonProperty("taskId") String taskId, - @JsonProperty("partitionStats") List partitionStats - ) + GeneratedPartitionsReport(String taskId, List partitionStats) { this.taskId = taskId; this.partitionStats = partitionStats; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 39760dc1dea1..0389cd0aaa1e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -308,7 +308,7 @@ PartialHashSegmentGenerateParallelIndexTaskRunner createPartialHashSegmentGenera @VisibleForTesting PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner( TaskToolbox toolbox, - List> ioConfigs + List ioConfigs ) { return new PartialHashSegmentMergeParallelIndexTaskRunner( @@ -493,7 +493,7 @@ private boolean useHashPartitions() private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throws Exception { // 1. Partial segment generation phase - ParallelIndexTaskRunner> indexingRunner = createRunner( + ParallelIndexTaskRunner indexingRunner = createRunner( toolbox, this::createPartialHashSegmentGenerateRunner ); @@ -508,7 +508,7 @@ private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throw // partition (interval, partitionId) -> partition locations Map, List> partitionToLocations = groupHashPartitionLocationsPerPartition(indexingRunner.getReports()); - final List> ioConfigs = createMergeIOConfigs( + final List ioConfigs = createHashMergeIOConfigs( ingestionSchema.getTuningConfig().getTotalNumMergeTasks(), partitionToLocations ); @@ -527,7 +527,7 @@ private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throw } private static Map, List> groupHashPartitionLocationsPerPartition( - Map> subTaskIdToReport + Map subTaskIdToReport ) { BiFunction createPartitionLocationFunction = @@ -546,13 +546,13 @@ private static Map, List> groupHa private static Map, List> groupPartitionLocationsPerPartition( - Map> subTaskIdToReport, + Map> subTaskIdToReport, BiFunction createPartitionLocationFunction ) { // partition (interval, partitionId) -> partition locations final Map, List> partitionToLocations = new HashMap<>(); - for (Entry> entry : subTaskIdToReport.entrySet()) { + for (Entry> entry : subTaskIdToReport.entrySet()) { final String subTaskId = entry.getKey(); final GeneratedPartitionsReport report = entry.getValue(); for (S partitionStat : report.getPartitionStats()) { @@ -567,9 +567,22 @@ Map, List> groupPartitionLocationsPerPartition( return partitionToLocations; } - private static List> createMergeIOConfigs( + private static List createHashMergeIOConfigs( int totalNumMergeTasks, - Map, List> partitionToLocations + Map, List> partitionToLocations + ) + { + return createMergeIOConfigs( + totalNumMergeTasks, + partitionToLocations, + PartialHashSegmentMergeIOConfig::new + ); + } + + private static List createMergeIOConfigs( + int totalNumMergeTasks, + Map, List> partitionToLocations, + Function, M> createPartialSegmentMergeIOConfig ) { final int numMergeTasks = Math.min(totalNumMergeTasks, partitionToLocations.size()); @@ -581,28 +594,28 @@ private static List ); // Randomly shuffle partitionIds to evenly distribute partitions of potentially different sizes // This will be improved once we collect partition stats properly. - // See HashPartitionStat in GeneratedPartitionsReport. + // See PartitionStat in GeneratedPartitionsReport. final List> partitions = new ArrayList<>(partitionToLocations.keySet()); Collections.shuffle(partitions, ThreadLocalRandom.current()); final int numPartitionsPerTask = (int) Math.round(partitions.size() / (double) numMergeTasks); - final List> assignedPartitionLocations = new ArrayList<>(numMergeTasks); + final List assignedPartitionLocations = new ArrayList<>(numMergeTasks); for (int i = 0; i < numMergeTasks - 1; i++) { - final List assignedToSameTask = partitions + final List assignedToSameTask = partitions .subList(i * numPartitionsPerTask, (i + 1) * numPartitionsPerTask) .stream() .flatMap(intervalAndPartitionId -> partitionToLocations.get(intervalAndPartitionId).stream()) .collect(Collectors.toList()); - assignedPartitionLocations.add(new PartialSegmentMergeIOConfig(assignedToSameTask)); + assignedPartitionLocations.add(createPartialSegmentMergeIOConfig.apply(assignedToSameTask)); } // The last task is assigned all remaining partitions. - final List assignedToSameTask = partitions + final List assignedToSameTask = partitions .subList((numMergeTasks - 1) * numPartitionsPerTask, partitions.size()) .stream() .flatMap(intervalAndPartitionId -> partitionToLocations.get(intervalAndPartitionId).stream()) .collect(Collectors.toList()); - assignedPartitionLocations.add(new PartialSegmentMergeIOConfig<>(assignedToSameTask)); + assignedPartitionLocations.add(createPartialSegmentMergeIOConfig.apply(assignedToSameTask)); return assignedPartitionLocations; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java index 32fb0b1ee572..50bef2ebcd11 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java @@ -34,7 +34,7 @@ * @see PartialHashSegmentMergeParallelIndexTaskRunner */ class PartialHashSegmentGenerateParallelIndexTaskRunner - extends FirehoseSplitParallelIndexTaskRunner> + extends FirehoseSplitParallelIndexTaskRunner { // For tests private final IndexTaskClientFactory taskClientFactory; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index a85f057dbb8b..9f7f4d65b5a1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -40,15 +40,16 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** * The worker task of {@link PartialHashSegmentGenerateParallelIndexTaskRunner}. This task partitions input data by - * hashing the segment granularity and partition dimensions in {@link - * org.apache.druid.indexer.partitions.PartitionsSpec}. Partitioned segments are stored in local storage using {@link - * org.apache.druid.indexing.worker.ShuffleDataSegmentPusher}. + * hashing the segment granularity and partition dimensions in {@link HashedPartitionsSpec}. Partitioned segments are + * stored in local storage using {@link org.apache.druid.indexing.worker.ShuffleDataSegmentPusher}. */ -public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask +public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask { public static final String TYPE = "partial_index_generate"; private static final String PROP_SPEC = "spec"; @@ -135,7 +136,15 @@ IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOE } @Override - HashPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment) + GeneratedHashPartitionsReport createGeneratedPartitionsReport(TaskToolbox toolbox, List segments) + { + List partitionStats = segments.stream() + .map(segment -> createPartitionStat(toolbox, segment)) + .collect(Collectors.toList()); + return new GeneratedHashPartitionsReport(getId(), partitionStats); + } + + private HashPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment) { return new HashPartitionStat( toolbox.getTaskExecutorNode().getHost(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java new file mode 100644 index 000000000000..927731fd311f --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.segment.indexing.IOConfig; + +import java.util.List; + +@JsonTypeName(PartialHashSegmentMergeTask.TYPE) +public class PartialHashSegmentMergeIOConfig extends PartialSegmentMergeIOConfig + implements IOConfig +{ + @JsonCreator + public PartialHashSegmentMergeIOConfig( + @JsonProperty("partitionLocations") List partitionLocations + ) + { + super(partitionLocations); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java new file mode 100644 index 000000000000..08ab59995bb2 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.indexing.DataSchema; + +public class PartialHashSegmentMergeIngestionSpec extends PartialSegmentMergeIngestionSpec +{ + @JsonCreator + public PartialHashSegmentMergeIngestionSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("ioConfig") PartialHashSegmentMergeIOConfig ioConfig, + @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig + ) + { + super(dataSchema, ioConfig, tuningConfig); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java index a5e45abe754e..16b54ab71ffb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java @@ -38,14 +38,14 @@ class PartialHashSegmentMergeParallelIndexTaskRunner extends ParallelIndexPhaseRunner { private final DataSchema dataSchema; - private final List> mergeIOConfigs; + private final List mergeIOConfigs; PartialHashSegmentMergeParallelIndexTaskRunner( TaskToolbox toolbox, String taskId, String groupId, DataSchema dataSchema, - List> mergeIOConfigs, + List mergeIOConfigs, ParallelIndexTuningConfig tuningConfig, Map context, IndexingServiceClient indexingServiceClient @@ -76,10 +76,10 @@ int getTotalNumSubTasks() } @VisibleForTesting - SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) + SubTaskSpec newTaskSpec(PartialHashSegmentMergeIOConfig ioConfig) { - final PartialSegmentMergeIngestionSpec ingestionSpec = - new PartialSegmentMergeIngestionSpec<>( + final PartialHashSegmentMergeIngestionSpec ingestionSpec = + new PartialHashSegmentMergeIngestionSpec( dataSchema, ioConfig, getTuningConfig() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java index c15af6474423..0e29eae973c8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java @@ -49,7 +49,7 @@ public class PartialHashSegmentMergeTask public static final String TYPE = "partial_index_merge"; private final HashedPartitionsSpec partitionsSpec; - private final PartialSegmentMergeIngestionSpec ingestionSchema; + private final PartialHashSegmentMergeIngestionSpec ingestionSchema; @JsonCreator public PartialHashSegmentMergeTask( @@ -59,7 +59,7 @@ public PartialHashSegmentMergeTask( @JsonProperty("resource") final TaskResource taskResource, @JsonProperty("supervisorTaskId") final String supervisorTaskId, @JsonProperty("numAttempts") final int numAttempts, // zero-based counting - @JsonProperty("spec") final PartialSegmentMergeIngestionSpec ingestionSchema, + @JsonProperty("spec") final PartialHashSegmentMergeIngestionSpec ingestionSchema, @JsonProperty("context") final Map context, @JacksonInject IndexingServiceClient indexingServiceClient, @JacksonInject IndexTaskClientFactory taskClientFactory, @@ -90,7 +90,7 @@ public PartialHashSegmentMergeTask( } @JsonProperty("spec") - private PartialSegmentMergeIngestionSpec getIngestionSchema() + private PartialHashSegmentMergeIngestionSpec getIngestionSchema() { return ingestionSchema; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index 58d433500857..d6a8d700704c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -55,12 +55,11 @@ import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; /** * Base class for parallel indexing perfect rollup worker partial segment generate tasks. */ -abstract class PartialSegmentGenerateTask extends PerfectRollupWorkerTask +abstract class PartialSegmentGenerateTask extends PerfectRollupWorkerTask { private final ParallelIndexIngestionSpec ingestionSchema; private final String supervisorTaskId; @@ -117,10 +116,7 @@ public final TaskStatus runTask(TaskToolbox toolbox) throws Exception ); final List segments = generateSegments(toolbox, firehoseFactory, firehoseTempDir); - final List partitionStats = segments.stream() - .map(segment -> createPartitionStat(toolbox, segment)) - .collect(Collectors.toList()); - taskClient.report(supervisorTaskId, new GeneratedPartitionsReport<>(getId(), partitionStats)); + taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments)); return TaskStatus.success(getId()); } @@ -131,9 +127,12 @@ public final TaskStatus runTask(TaskToolbox toolbox) throws Exception abstract IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException; /** - * @return {@link PartitionStat} suitable for the desired segment partitioning strategy. + * @return {@link GeneratedPartitionsReport} suitable for the desired segment partitioning strategy. */ - abstract T createPartitionStat(TaskToolbox toolbox, DataSegment segment); + abstract T createGeneratedPartitionsReport( + TaskToolbox toolbox, + List segments + ); private List generateSegments( final TaskToolbox toolbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java index 4d259f60acf7..50e3a09c2238 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java @@ -19,21 +19,17 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import org.apache.druid.segment.indexing.IOConfig; import java.util.List; -@JsonTypeName(PartialHashSegmentMergeTask.TYPE) -public class PartialSegmentMergeIOConfig implements IOConfig +abstract class PartialSegmentMergeIOConfig implements IOConfig { private final List partitionLocations; - @JsonCreator - public PartialSegmentMergeIOConfig(@JsonProperty("partitionLocations") List partitionLocations) + PartialSegmentMergeIOConfig(List partitionLocations) { Preconditions.checkState( partitionLocations != null && !partitionLocations.isEmpty(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java index 67d4868c28e5..b0ea81d8fd31 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java @@ -19,19 +19,16 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; -public class PartialSegmentMergeIngestionSpec - extends IngestionSpec, ParallelIndexTuningConfig> +abstract class PartialSegmentMergeIngestionSpec + extends IngestionSpec { - @JsonCreator - public PartialSegmentMergeIngestionSpec( - @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") PartialSegmentMergeIOConfig ioConfig, - @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig + PartialSegmentMergeIngestionSpec( + DataSchema dataSchema, + T ioConfig, + ParallelIndexTuningConfig tuningConfig ) { super(dataSchema, ioConfig, tuningConfig); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java index ec26cc8c1103..e60423533dfa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java @@ -30,7 +30,7 @@ @JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = PushedSegmentsReport.class) @JsonSubTypes(value = { @Type(name = PushedSegmentsReport.TYPE, value = PushedSegmentsReport.class), - @Type(name = GeneratedPartitionsReport.TYPE, value = GeneratedPartitionsReport.class) + @Type(name = GeneratedHashPartitionsReport.TYPE, value = GeneratedHashPartitionsReport.class) }) public interface SubTaskReport { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReportTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java similarity index 80% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReportTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java index 67132f39280f..6e4bcd2be383 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReportTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.segment.TestHelper; import org.junit.Before; @@ -27,16 +26,16 @@ import java.util.Collections; -public class GeneratedPartitionsReportTest +public class GeneratedHashPartitionsReportTest { private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); - private GeneratedPartitionsReport target; + private GeneratedHashPartitionsReport target; @Before public void setup() { - target = new GeneratedPartitionsReport<>( + target = new GeneratedHashPartitionsReport( "task-id", Collections.singletonList( new HashPartitionStat( @@ -55,12 +54,6 @@ public void setup() @Test public void serializesDeserializes() { - TestHelper.testSerializesDeserializes( - OBJECT_MAPPER, - target, - new TypeReference>() - { - } - ); + TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java index 248a7e3d5ded..3116fbba37da 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -189,7 +189,7 @@ public PartialHashSegmentGenerateParallelIndexTaskRunner createPartialHashSegmen @Override public PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner( TaskToolbox toolbox, - List> ioConfigs + List ioConfigs ) { return new TestPartialHashSegmentMergeParallelIndexTaskRunner( @@ -230,7 +230,7 @@ private static class TestPartialHashSegmentMergeParallelIndexTaskRunner private TestPartialHashSegmentMergeParallelIndexTaskRunner( TaskToolbox toolbox, ParallelIndexSupervisorTask supervisorTask, - List> mergeIOConfigs, + List mergeIOConfigs, IndexingServiceClient indexingServiceClient ) { @@ -248,10 +248,10 @@ private TestPartialHashSegmentMergeParallelIndexTaskRunner( } @Override - SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) + SubTaskSpec newTaskSpec(PartialHashSegmentMergeIOConfig ioConfig) { - final PartialSegmentMergeIngestionSpec ingestionSpec = - new PartialSegmentMergeIngestionSpec<>( + final PartialHashSegmentMergeIngestionSpec ingestionSpec = + new PartialHashSegmentMergeIngestionSpec( supervisorTask.getIngestionSchema().getDataSchema(), ioConfig, getTuningConfig() @@ -294,7 +294,7 @@ private TestPartialHashSegmentMergeTask( TaskResource taskResource, String supervisorTaskId, int numAttempts, - PartialSegmentMergeIngestionSpec ingestionSchema, + PartialHashSegmentMergeIngestionSpec ingestionSchema, Map context, IndexingServiceClient indexingServiceClient, IndexTaskClientFactory taskClientFactory, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java new file mode 100644 index 000000000000..b16c77eb2354 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.segment.TestHelper; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +public class PartialHashSegmentMergeIOConfigTest +{ + private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( + Factory.HOST, + Factory.PORT, + Factory.USE_HTTPS, + Factory.SUBTASK_ID, + Factory.INTERVAL, + Factory.PARTITION_ID + ); + + private PartialHashSegmentMergeIOConfig target; + + @Before + public void setup() + { + target = new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); + } + + @Test + public void serializesDeserializes() + { + TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java new file mode 100644 index 000000000000..e82f45fe26b8 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.segment.TestHelper; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +public class PartialHashSegmentMergeIngestionSpecTest +{ + private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( + Factory.HOST, + Factory.PORT, + Factory.USE_HTTPS, + Factory.SUBTASK_ID, + Factory.INTERVAL, + Factory.PARTITION_ID + ); + private static final PartialHashSegmentMergeIOConfig IO_CONFIG = + new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); + private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( + null, + 1, + Collections.emptyList() + ); + + private PartialHashSegmentMergeIngestionSpec target; + + @Before + public void setup() + { + target = new PartialHashSegmentMergeIngestionSpec( + Factory.createDataSchema(Factory.INPUT_INTERVALS), + IO_CONFIG, + new Factory.TuningConfigBuilder() + .partitionsSpec(PARTITIONS_SPEC) + .build() + ); + } + + @Test + public void serializesDeserializes() + { + TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java index 669a18b8c079..324befa3bce7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java @@ -40,15 +40,15 @@ public class PartialHashSegmentMergeTaskTest Factory.INTERVAL, Factory.PARTITION_ID ); - private static final PartialSegmentMergeIOConfig IO_CONFIG = - new PartialSegmentMergeIOConfig<>(Collections.singletonList(HASH_PARTITION_LOCATION)); + private static final PartialHashSegmentMergeIOConfig IO_CONFIG = + new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( null, 1, Collections.emptyList() ); - private static final PartialSegmentMergeIngestionSpec INGESTION_SPEC = - new PartialSegmentMergeIngestionSpec<>( + private static final PartialHashSegmentMergeIngestionSpec INGESTION_SPEC = + new PartialHashSegmentMergeIngestionSpec( Factory.createDataSchema(Factory.INPUT_INTERVALS), IO_CONFIG, new Factory.TuningConfigBuilder() diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 1d7e5faa28ec..8264767e601e 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -19,7 +19,6 @@ package org.apache.druid.segment; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; @@ -46,7 +45,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -400,50 +398,10 @@ public static void testSerializesDeserializes(Object object) } public static void testSerializesDeserializes(ObjectMapper objectMapper, Object object) - { - testSerializesDeserializes( - objectMapper, - object, - serialized -> { - try { - return objectMapper.readValue(serialized, object.getClass()); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - ); - } - - public static void testSerializesDeserializes( - ObjectMapper objectMapper, - Object object, - TypeReference typeReference - ) - { - testSerializesDeserializes( - objectMapper, - object, - serialized -> { - try { - return objectMapper.readValue(serialized, typeReference); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - ); - } - - private static void testSerializesDeserializes( - ObjectMapper objectMapper, - Object object, - Function readValueFunction - ) { try { String serialized = objectMapper.writeValueAsString(object); - Object deserialized = readValueFunction.apply(serialized); + Object deserialized = objectMapper.readValue(serialized, object.getClass()); Assert.assertEquals(serialized, objectMapper.writeValueAsString(deserialized)); } catch (IOException e) { From db78ea4ab4ce26f26063cece2d64d03b0032cad4 Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Tue, 12 Nov 2019 10:20:17 -0800 Subject: [PATCH 06/12] Fix TeamCity inspection errors/warnings --- .../org/apache/druid/indexer/partitions/PartitionsSpec.java | 4 ---- .../indexing/common/task/CachingLocalSegmentAllocator.java | 1 - .../task/batch/parallel/GeneratedHashPartitionsReport.java | 4 ++-- .../task/batch/parallel/PartialHashSegmentMergeIOConfig.java | 4 ++-- .../batch/parallel/PartialHashSegmentMergeIngestionSpec.java | 4 ++-- .../iterator/DefaultIndexTaskInputRowIteratorBuilder.java | 4 +--- 6 files changed, 7 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java b/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java index 3c82ed49349a..61bd66bc9a7c 100644 --- a/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java +++ b/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java @@ -79,8 +79,6 @@ static Integer resolveHistoricalNullIfNeeded(@Nullable Integer val) /** * @return True if this partitionSpec's type is compatible with forceGuaranteedRollup=true. - * - * @see org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig#forceGuaranteedRollup */ @JsonIgnore default boolean isForceGuaranteedRollupCompatibleType() @@ -90,8 +88,6 @@ default boolean isForceGuaranteedRollupCompatibleType() /** * @return True if this partitionSpec's property values are compatible with forceGuaranteedRollup=true. - * - * @see org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig#forceGuaranteedRollup */ @JsonIgnore default boolean isForceGuaranteedRollupCompatible() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index c9a87bc26cc5..4642ff336d12 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java @@ -43,7 +43,6 @@ * Allocates all necessary segments locally at the beginning and reuses them. * * @see DefaultCachingLocalSegmentAllocator - * @see RangePartitionCachingLocalSegmentAllocator */ class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java index 27266031d43e..85574b7b254b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java @@ -29,12 +29,12 @@ * This report is collected by {@link ParallelIndexSupervisorTask} and * used to generate {@link PartialHashSegmentMergeIOConfig}. */ -public class GeneratedHashPartitionsReport extends GeneratedPartitionsReport implements SubTaskReport +class GeneratedHashPartitionsReport extends GeneratedPartitionsReport implements SubTaskReport { public static final String TYPE = "generated_partitions"; @JsonCreator - public GeneratedHashPartitionsReport( + GeneratedHashPartitionsReport( @JsonProperty("taskId") String taskId, @JsonProperty("partitionStats") List partitionStats ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java index 927731fd311f..2bc00ce3d9b6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java @@ -27,11 +27,11 @@ import java.util.List; @JsonTypeName(PartialHashSegmentMergeTask.TYPE) -public class PartialHashSegmentMergeIOConfig extends PartialSegmentMergeIOConfig +class PartialHashSegmentMergeIOConfig extends PartialSegmentMergeIOConfig implements IOConfig { @JsonCreator - public PartialHashSegmentMergeIOConfig( + PartialHashSegmentMergeIOConfig( @JsonProperty("partitionLocations") List partitionLocations ) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java index 08ab59995bb2..abfef7608809 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.segment.indexing.DataSchema; -public class PartialHashSegmentMergeIngestionSpec extends PartialSegmentMergeIngestionSpec +class PartialHashSegmentMergeIngestionSpec extends PartialSegmentMergeIngestionSpec { @JsonCreator - public PartialHashSegmentMergeIngestionSpec( + PartialHashSegmentMergeIngestionSpec( @JsonProperty("dataSchema") DataSchema dataSchema, @JsonProperty("ioConfig") PartialHashSegmentMergeIOConfig ioConfig, @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java index 22b621d968a9..41e56363a294 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java @@ -52,8 +52,6 @@ * If any of the handlers invoke their respective callback, the {@link InputRowIterator} will yield * a null {@link InputRow} next; otherwise, the next {@link InputRow} is yielded. * - * - * @see RangePartitionIndexTaskInputRowIteratorBuilder */ public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRowIteratorBuilder { @@ -61,7 +59,7 @@ public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRo private GranularitySpec granularitySpec = null; private InputRowIterator.InputRowHandler nullRowHandler = null; private InputRowIterator.InputRowHandler absentBucketIntervalHandler = null; - private List appendedInputRowHandlers = new ArrayList(); + private List appendedInputRowHandlers = new ArrayList<>(); @Override public DefaultIndexTaskInputRowIteratorBuilder firehose(Firehose firehose) From dd2731f5fb4ad54e9c405b0791bb406c3010910e Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Tue, 12 Nov 2019 13:22:18 -0800 Subject: [PATCH 07/12] Fix TeamCity inspection errors/warnings again --- .../org/apache/druid/indexer/partitions/PartitionsSpec.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java b/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java index 61bd66bc9a7c..21cdec7e6e69 100644 --- a/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java +++ b/core/src/main/java/org/apache/druid/indexer/partitions/PartitionsSpec.java @@ -98,8 +98,6 @@ default boolean isForceGuaranteedRollupCompatible() /** * @return Message describing why this partitionSpec is incompatible with forceGuaranteedRollup=true. Empty string if * the partitionSpec is compatible. - * - * @see org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig#forceGuaranteedRollup */ @JsonIgnore String getForceGuaranteedRollupIncompatiblityReason(); From 3ea47c6677cf6bed9767cefd08a432714692ecfc Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Fri, 15 Nov 2019 13:44:28 -0800 Subject: [PATCH 08/12] Integrate changes with those from #8823 --- ...tor.java => HandlingInputRowIterator.java} | 38 +- ...java => HandlingInputRowIteratorTest.java} | 41 +- .../common/task/InputSourceProcessor.java | 28 +- .../FirehoseSplitParallelIndexTaskRunner.java | 26 - ...faultIndexTaskInputRowIteratorBuilder.java | 32 +- .../IndexTaskInputRowIteratorBuilder.java | 9 +- ...bstractMultiPhaseParallelIndexingTest.java | 159 +++-- ...rtitionMultiPhaseParallelIndexingTest.java | 15 +- .../MultiPhaseParallelIndexingTest.java | 593 ------------------ ...tIndexTaskInputRowIteratorBuilderTest.java | 40 +- .../task/batch/parallel/iterator/Factory.java | 49 +- 11 files changed, 209 insertions(+), 821 deletions(-) rename core/src/main/java/org/apache/druid/data/input/{InputRowIterator.java => HandlingInputRowIterator.java} (72%) rename core/src/test/java/org/apache/druid/data/input/{InputRowIteratorTest.java => HandlingInputRowIteratorTest.java} (71%) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowIterator.java b/core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java similarity index 72% rename from core/src/main/java/org/apache/druid/data/input/InputRowIterator.java rename to core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java index a6c526b4b478..238ba3d69c8b 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowIterator.java +++ b/core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java @@ -19,17 +19,16 @@ package org.apache.druid.data.input; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + import javax.annotation.Nullable; -import java.io.IOException; -import java.io.UncheckedIOException; import java.util.Iterator; import java.util.List; -import java.util.NoSuchElementException; /** - * {@link Iterator} for {@link InputRow}s from a {@link Firehose}. + * Decorated {@link CloseableIterator} that can process rows with {@link InputRowHandler}s. */ -public class InputRowIterator implements Iterator +public class HandlingInputRowIterator implements Iterator { @FunctionalInterface public interface InputRowHandler @@ -40,33 +39,28 @@ public interface InputRowHandler boolean handle(InputRow inputRow); } - private final Firehose firehose; + private final CloseableIterator delegate; private final List inputRowHandlers; /** - * @param firehose Source of {@link InputRow}s + * @param inputRowIterator Source of {@link InputRow}s * @param inputRowHandlers Before yielding the next {@link InputRow}, each {@link InputRowHandler} is sequentially * applied to the {@link InputRow} until one of them returns true or all of the handlers are * applied. */ - public InputRowIterator( - Firehose firehose, + public HandlingInputRowIterator( + CloseableIterator inputRowIterator, List inputRowHandlers ) { - this.firehose = firehose; + this.delegate = inputRowIterator; this.inputRowHandlers = inputRowHandlers; } @Override public boolean hasNext() { - try { - return firehose.hasMore(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } + return delegate.hasNext(); } /** @@ -76,17 +70,7 @@ public boolean hasNext() @Nullable public InputRow next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - InputRow inputRow; - try { - inputRow = firehose.nextRow(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } + InputRow inputRow = delegate.next(); for (InputRowHandler inputRowHandler : inputRowHandlers) { if (inputRowHandler.handle(inputRow)) { diff --git a/core/src/test/java/org/apache/druid/data/input/InputRowIteratorTest.java b/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java similarity index 71% rename from core/src/test/java/org/apache/druid/data/input/InputRowIteratorTest.java rename to core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java index 52cd8fffe435..88ddd5c0dff4 100644 --- a/core/src/test/java/org/apache/druid/data/input/InputRowIteratorTest.java +++ b/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.data.input; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; @@ -34,32 +35,32 @@ import java.util.NoSuchElementException; @RunWith(Enclosed.class) -public class InputRowIteratorTest +public class HandlingInputRowIteratorTest { public static class AbsentRowTest { - private static final Firehose EMPTY_FIREHOSE = new TestFirehose() + private static final CloseableIterator EMPTY_ITERATOR = new TestCloseableIterator() { @Override - public boolean hasMore() + public boolean hasNext() { return false; } @Nullable @Override - public InputRow nextRow() + public InputRow next() { - return null; + throw new NoSuchElementException(); } }; - private InputRowIterator target; + private HandlingInputRowIterator target; @Before public void setup() { - target = new InputRowIterator(EMPTY_FIREHOSE, Collections.emptyList()); + target = new HandlingInputRowIterator(EMPTY_ITERATOR, Collections.emptyList()); } @Test @@ -94,7 +95,7 @@ public void setup() @Test public void hasNext() { - InputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); + HandlingInputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); Assert.assertTrue(target.hasNext()); Assert.assertFalse(unsuccessfulHandler.invoked); } @@ -102,7 +103,7 @@ public void hasNext() @Test public void yieldsNextIfUnhandled() { - InputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); + HandlingInputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); Assert.assertEquals(INPUT_ROW1, target.next()); Assert.assertTrue(unsuccessfulHandler.invoked); } @@ -110,7 +111,7 @@ public void yieldsNextIfUnhandled() @Test public void yieldsNullIfHandledByFirst() { - InputRowIterator target = createInputRowIterator(successfulHandler, unsuccessfulHandler); + HandlingInputRowIterator target = createInputRowIterator(successfulHandler, unsuccessfulHandler); Assert.assertNull(target.next()); Assert.assertTrue(successfulHandler.invoked); Assert.assertFalse(unsuccessfulHandler.invoked); @@ -119,39 +120,39 @@ public void yieldsNullIfHandledByFirst() @Test public void yieldsNullIfHandledBySecond() { - InputRowIterator target = createInputRowIterator(unsuccessfulHandler, successfulHandler); + HandlingInputRowIterator target = createInputRowIterator(unsuccessfulHandler, successfulHandler); Assert.assertNull(target.next()); Assert.assertTrue(unsuccessfulHandler.invoked); Assert.assertTrue(successfulHandler.invoked); } - private static InputRowIterator createInputRowIterator( - InputRowIterator.InputRowHandler firstHandler, - InputRowIterator.InputRowHandler secondHandler + private static HandlingInputRowIterator createInputRowIterator( + HandlingInputRowIterator.InputRowHandler firstHandler, + HandlingInputRowIterator.InputRowHandler secondHandler ) { - Firehose firehose = new TestFirehose() + CloseableIterator iterator = new TestCloseableIterator() { private final Iterator delegate = INPUT_ROWS.iterator(); @Override - public boolean hasMore() + public boolean hasNext() { return delegate.hasNext(); } @Nullable @Override - public InputRow nextRow() + public InputRow next() { return delegate.next(); } }; - return new InputRowIterator(firehose, Arrays.asList(firstHandler, secondHandler)); + return new HandlingInputRowIterator(iterator, Arrays.asList(firstHandler, secondHandler)); } - private static class TestInputRowHandler implements InputRowIterator.InputRowHandler + private static class TestInputRowHandler implements HandlingInputRowIterator.InputRowHandler { boolean invoked = false; @@ -171,7 +172,7 @@ public boolean handle(InputRow inputRow) } } - private abstract static class TestFirehose implements Firehose + private abstract static class TestCloseableIterator implements CloseableIterator { @Override public void close() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index 45946f8995fa..5286e0729510 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -20,9 +20,9 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Optional; +import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowIterator; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; @@ -104,17 +104,6 @@ public SegmentsAndMetadata process( : null; final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); - try ( - final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) - ) { - InputRowIterator iterator = inputRowIteratorBuilder - .firehose(firehose) - .granularitySpec(granularitySpec) - .nullRowRunnable(buildSegmentsMeters::incrementThrownAway) - .absentBucketIntervalConsumer(inputRow -> buildSegmentsMeters.incrementThrownAway()) - .build(); - - while (iterator.hasNext()) { final List metricsNames = Arrays.stream(dataSchema.getAggregators()) .map(AggregatorFactory::getName) .collect(Collectors.toList()); @@ -130,13 +119,16 @@ public SegmentsAndMetadata process( ) ); try (final CloseableIterator inputRowIterator = inputSourceReader.read()) { - while (inputRowIterator.hasNext()) { - try { - final InputRow inputRow = firehose.nextRow(); - - InputRow inputRow = iterator.next(); - final InputRow inputRow = inputRowIterator.next(); + HandlingInputRowIterator iterator = inputRowIteratorBuilder + .delegate(inputRowIterator) + .granularitySpec(granularitySpec) + .nullRowRunnable(buildSegmentsMeters::incrementThrownAway) + .absentBucketIntervalConsumer(inputRow -> buildSegmentsMeters.incrementThrownAway()) + .build(); + while (iterator.hasNext()) { + try { + final InputRow inputRow = iterator.next(); if (inputRow == null) { continue; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java index 336b42ce250f..f58b96e0eb80 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java @@ -82,33 +82,7 @@ final int getTotalNumSubTasks() throws IOException ); } - @VisibleForTesting - ParallelIndexIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - - @VisibleForTesting - FiniteFirehoseFactory getBaseFirehoseFactory() - { - return baseFirehoseFactory; - } - - SubTaskSpec newTaskSpec(InputSplit split) final SubTaskSpec newTaskSpec(InputSplit split) - @VisibleForTesting - ParallelIndexIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - - @VisibleForTesting - SplittableInputSource getBaseInputSource() - { - return baseInputSource; - } - - SubTaskSpec newTaskSpec(InputSplit split) { final FirehoseFactory firehoseFactory; final SplittableInputSource inputSource; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java index 41e56363a294..b03bbd4d466f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java @@ -22,12 +22,12 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowIterator; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.joda.time.Interval; @@ -38,7 +38,7 @@ /** *
- * Build a default {@link InputRowIterator} for {@link IndexTask}s. Each {@link InputRow} is
+ * Build a default {@link HandlingInputRowIterator} for {@link IndexTask}s. Each {@link InputRow} is
  * processed by the following handlers, in order:
  *
  *   1. Null row: If {@link InputRow} is null, invoke the null row {@link Runnable} callback.
@@ -49,22 +49,22 @@
  *      {@link GranularitySpec} bucket intervals, invoke the absent bucket interval {@link Consumer}
  *      callback.
  *
- * If any of the handlers invoke their respective callback, the {@link InputRowIterator} will yield
+ * If any of the handlers invoke their respective callback, the {@link HandlingInputRowIterator} will yield
  * a null {@link InputRow} next; otherwise, the next {@link InputRow} is yielded.
  * 
*/ public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRowIteratorBuilder { - private Firehose firehose = null; + private CloseableIterator delegate = null; private GranularitySpec granularitySpec = null; - private InputRowIterator.InputRowHandler nullRowHandler = null; - private InputRowIterator.InputRowHandler absentBucketIntervalHandler = null; - private List appendedInputRowHandlers = new ArrayList<>(); + private HandlingInputRowIterator.InputRowHandler nullRowHandler = null; + private HandlingInputRowIterator.InputRowHandler absentBucketIntervalHandler = null; + private List appendedInputRowHandlers = new ArrayList<>(); @Override - public DefaultIndexTaskInputRowIteratorBuilder firehose(Firehose firehose) + public DefaultIndexTaskInputRowIteratorBuilder delegate(CloseableIterator inputRowIterator) { - this.firehose = firehose; + this.delegate = inputRowIterator; return this; } @@ -105,32 +105,32 @@ public DefaultIndexTaskInputRowIteratorBuilder absentBucketIntervalConsumer( } @Override - public InputRowIterator build() + public HandlingInputRowIterator build() { - Preconditions.checkNotNull(firehose, "firehose required"); + Preconditions.checkNotNull(delegate, "delegate required"); Preconditions.checkNotNull(granularitySpec, "granularitySpec required"); Preconditions.checkNotNull(nullRowHandler, "nullRowRunnable required"); Preconditions.checkNotNull(absentBucketIntervalHandler, "absentBucketIntervalConsumer required"); - ImmutableList.Builder handlersBuilder = ImmutableList.builder() + ImmutableList.Builder handlersBuilder = ImmutableList.builder() .add(nullRowHandler) .add(createInvalidTimestampHandler()) .add(absentBucketIntervalHandler) .addAll(appendedInputRowHandlers); - return new InputRowIterator(firehose, handlersBuilder.build()); + return new HandlingInputRowIterator(delegate, handlersBuilder.build()); } /** * @param inputRowHandler Optionally, append this input row handler to the required ones. */ - DefaultIndexTaskInputRowIteratorBuilder appendInputRowHandler(InputRowIterator.InputRowHandler inputRowHandler) + DefaultIndexTaskInputRowIteratorBuilder appendInputRowHandler(HandlingInputRowIterator.InputRowHandler inputRowHandler) { this.appendedInputRowHandlers.add(inputRowHandler); return this; } - private InputRowIterator.InputRowHandler createInvalidTimestampHandler() + private HandlingInputRowIterator.InputRowHandler createInvalidTimestampHandler() { return inputRow -> { if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java index b8842e7ea9c8..80b4ea88615b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java @@ -20,8 +20,9 @@ package org.apache.druid.indexing.common.task.batch.parallel.iterator; import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowIterator; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import java.util.function.Consumer; @@ -35,9 +36,9 @@ public interface IndexTaskInputRowIteratorBuilder }; /** - * @param firehose Source of {@link InputRow}s. + * @param inputRowIterator Source of {@link InputRow}s. */ - IndexTaskInputRowIteratorBuilder firehose(Firehose firehose); + IndexTaskInputRowIteratorBuilder delegate(CloseableIterator inputRowIterator); /** * @param granularitySpec {@link GranularitySpec} for the {@link org.apache.druid.segment.indexing.DataSchema} @@ -56,5 +57,5 @@ public interface IndexTaskInputRowIteratorBuilder */ IndexTaskInputRowIteratorBuilder absentBucketIntervalConsumer(Consumer absentBucketIntervalConsumer); - InputRowIterator build(); + HandlingInputRowIterator build(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 04ce63db4c7e..c45c5ce09212 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.common.task.batch.parallel; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexer.TaskState; @@ -31,7 +31,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -48,9 +47,11 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.Segment; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.After; @@ -78,10 +79,12 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn ); private final LockGranularity lockGranularity; + private final boolean useInputFormatApi; - AbstractMultiPhaseParallelIndexingTest(LockGranularity lockGranularity) + AbstractMultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) { this.lockGranularity = lockGranularity; + this.useInputFormatApi = useInputFormatApi; } @Before @@ -102,14 +105,16 @@ public void teardown() Set runTestTask( ParseSpec parseSpec, Interval interval, - FirehoseFactory firehoseFactory, + File inputDir, + String filter, DimensionBasedPartitionsSpec partitionsSpec ) throws Exception { final ParallelIndexSupervisorTask task = newTask( parseSpec, interval, - new ParallelIndexIOConfig(firehoseFactory, false), + inputDir, + filter, partitionsSpec ); @@ -130,78 +135,94 @@ Set runTestTask( private ParallelIndexSupervisorTask newTask( ParseSpec parseSpec, Interval interval, - ParallelIndexIOConfig ioConfig, + File inputDir, + String filter, DimensionBasedPartitionsSpec partitionsSpec ) { - return newTask( - parseSpec, - interval, + GranularitySpec granularitySpec = new UniformGranularitySpec( Granularities.DAY, - ioConfig, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - partitionsSpec, - null, - null, - null, - true, - null, - null, - null, - null, - 2, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) ); - } - private ParallelIndexSupervisorTask newTask( - ParseSpec parseSpec, - Interval interval, - Granularity segmentGranularity, - ParallelIndexIOConfig ioConfig, - ParallelIndexTuningConfig tuningConfig - ) - { - // set up ingestion spec - //noinspection unchecked - final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser(parseSpec, null), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null, - getObjectMapper() - ), - ioConfig, - tuningConfig + ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, + null, + null, + null, + null, + null, + null, + partitionsSpec, + null, + null, + null, + true, + null, + null, + null, + null, + 2, + null, + null, + null, + null, + null, + null, + null, + null, + null ); + final ParallelIndexIngestionSpec ingestionSpec; + + if (useInputFormatApi) { + ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( + null, + new LocalInputSource(inputDir, filter), + parseSpec.toInputFormat(), + false + ); + ingestionSpec = new ParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + parseSpec.getTimestampSpec(), + parseSpec.getDimensionsSpec(), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + granularitySpec, + null + ), + ioConfig, + tuningConfig + ); + } else { + ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( + new LocalFirehoseFactory(inputDir, filter, null), + false + ); + //noinspection unchecked + ingestionSpec = new ParallelIndexIngestionSpec( + new DataSchema( + "dataSource", + getObjectMapper().convertValue( + new StringInputRowParser(parseSpec, null), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + granularitySpec, + null, + getObjectMapper() + ), + ioConfig, + tuningConfig + ); + } + // set up test tools return createParallelIndexSupervisorTask( null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java index 3116fbba37da..dffd9d52e8b2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -36,7 +36,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.scan.ScanResultValue; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.joda.time.Interval; @@ -79,20 +78,21 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh 0 ); - @Parameterized.Parameters(name = "{0}") + @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK}, - new Object[]{LockGranularity.SEGMENT} + new Object[]{LockGranularity.TIME_CHUNK, false}, + new Object[]{LockGranularity.TIME_CHUNK, true}, + new Object[]{LockGranularity.SEGMENT, true} ); } private File inputDir; - public HashPartitionMultiPhaseParallelIndexingTest(LockGranularity lockGranularity) + public HashPartitionMultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) { - super(lockGranularity); + super(lockGranularity, useInputFormatApi); } @Override @@ -127,7 +127,8 @@ public void testRun() throws Exception final Set publishedSegments = runTestTask( PARSE_SPEC, Intervals.of("2017/2018"), - new LocalFirehoseFactory(inputDir, "test_*", null), + inputDir, + "test_*", new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")) ); assertHashedPartition(publishedSegments); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java deleted file mode 100644 index b543247697f5..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java +++ /dev/null @@ -1,593 +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 org.apache.druid.indexing.common.task.batch.parallel; - -import com.google.common.collect.ImmutableList; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.indexing.common.SegmentLoaderFactory; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.common.task.TestAppenderatorsManager; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.query.DefaultGenericQueryMetricsFactory; -import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.query.scan.ScanQueryConfig; -import org.apache.druid.query.scan.ScanQueryEngine; -import org.apache.druid.query.scan.ScanQueryQueryToolChest; -import org.apache.druid.query.scan.ScanQueryRunnerFactory; -import org.apache.druid.query.scan.ScanResultValue; -import org.apache.druid.query.spec.SpecificSegmentSpec; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.SegmentLoader; -import org.apache.druid.segment.loading.SegmentLoadingException; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -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.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -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.ArrayList; -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.Set; - -@RunWith(Parameterized.class) -public class MultiPhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest -{ - @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") - public static Iterable constructorFeeder() - { - return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK, false}, - new Object[]{LockGranularity.TIME_CHUNK, true}, - new Object[]{LockGranularity.SEGMENT, true} - ); - } - - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - private final LockGranularity lockGranularity; - private final boolean useInputFormatApi; - - private File inputDir; - - public MultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) - { - this.lockGranularity = lockGranularity; - this.useInputFormatApi = useInputFormatApi; - } - - @Before - public void setup() throws IOException - { - inputDir = temporaryFolder.newFolder("data"); - // set up data - for (int i = 0; i < 10; i++) { - try (final Writer writer = - Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { - for (int j = 0; j < 10; j++) { - writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", j + 1, i + 10, i)); - writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", j + 2, i + 11, 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,%d th test file\n", i + 1, i + 10, i)); - } - } - - indexingServiceClient = new LocalIndexingServiceClient(); - localDeepStorage = temporaryFolder.newFolder("localStorage"); - initializeIntermeidaryDataManager(); - } - - @After - public void teardown() - { - indexingServiceClient.shutdown(); - temporaryFolder.delete(); - } - - @Test - public void testRun() throws Exception - { - final Set publishedSegments = runTestTask( - Intervals.of("2017/2018"), - new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")) - ); - assertHashedPartition(publishedSegments); - } - - private Set runTestTask(Interval interval, HashedPartitionsSpec partitionsSpec) throws Exception - { - final ParallelIndexSupervisorTask task = newTask(interval, partitionsSpec); - actionClient = createActionClient(task); - toolbox = createTaskToolbox(task); - - prepareTaskForLocking(task); - task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); - Assert.assertTrue(task.isReady(actionClient)); - Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); - shutdownTask(task); - return actionClient.getPublishedSegments(); - } - - private ParallelIndexSupervisorTask newTask( - Interval interval, - HashedPartitionsSpec partitionsSpec - ) - { - return newTask( - interval, - Granularities.DAY, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - partitionsSpec, - null, - null, - null, - true, - null, - null, - null, - null, - 2, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) - ); - } - - private ParallelIndexSupervisorTask newTask( - Interval interval, - Granularity segmentGranularity, - ParallelIndexTuningConfig tuningConfig - ) - { - final ParseSpec parseSpec = new CSVParseSpec( - new TimestampSpec("ts", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2"))), - null, - Arrays.asList("ts", "dim1", "dim2", "val"), - false, - 0 - ); - final ParallelIndexIngestionSpec ingestionSpec; - if (useInputFormatApi) { - ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - parseSpec.getTimestampSpec(), - parseSpec.getDimensionsSpec(), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), - new ParallelIndexIOConfig( - null, - new LocalInputSource(inputDir, "test_*"), - parseSpec.toInputFormat(), - false - ), - tuningConfig - ); - } else { - ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - getObjectMapper().convertValue( - new StringInputRowParser(parseSpec, null), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null, - getObjectMapper() - ), - new ParallelIndexIOConfig(new LocalFirehoseFactory(inputDir, "test_*", null), false), - tuningConfig - ); - } - - // set up test tools - return new TestSupervisorTask( - null, - null, - ingestionSpec, - new HashMap<>(), - indexingServiceClient - ); - } - - private void assertHashedPartition(Set publishedSegments) throws IOException, SegmentLoadingException - { - final Map> intervalToSegments = new HashMap<>(); - publishedSegments.forEach( - segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) - ); - final File tempSegmentDir = temporaryFolder.newFolder(); - for (List segmentsInInterval : intervalToSegments.values()) { - Assert.assertEquals(2, segmentsInInterval.size()); - for (DataSegment segment : segmentsInInterval) { - final SegmentLoader loader = new SegmentLoaderFactory(getIndexIO(), getObjectMapper()) - .manufacturate(tempSegmentDir); - ScanQueryRunnerFactory factory = new ScanQueryRunnerFactory( - new ScanQueryQueryToolChest( - new ScanQueryConfig().setLegacy(false), - DefaultGenericQueryMetricsFactory.instance() - ), - new ScanQueryEngine(), - new ScanQueryConfig() - ); - final QueryRunner runner = factory.createRunner(loader.getSegment(segment)); - final List results = runner.run( - QueryPlus.wrap( - new ScanQuery( - new TableDataSource("dataSource"), - new SpecificSegmentSpec( - new SegmentDescriptor( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() - ) - ), - null, - null, - 0, - 0, - null, - null, - ImmutableList.of("dim1", "dim2"), - false, - null - ) - ) - ).toList(); - final int hash = HashBasedNumberedShardSpec.hash(getObjectMapper(), (List) results.get(0).getEvents()); - for (ScanResultValue value : results) { - Assert.assertEquals( - hash, - HashBasedNumberedShardSpec.hash(getObjectMapper(), (List) value.getEvents()) - ); - } - } - } - } - - private static class TestSupervisorTask extends TestParallelIndexSupervisorTask - { - TestSupervisorTask( - String id, - TaskResource taskResource, - ParallelIndexIngestionSpec ingestionSchema, - Map context, - IndexingServiceClient indexingServiceClient - ) - { - super(id, taskResource, ingestionSchema, context, indexingServiceClient); - } - - @Override - public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerateRunner(TaskToolbox toolbox) - { - return new TestPartialSegmentGenerateRunner(toolbox, this, getIndexingServiceClient()); - } - - @Override - public PartialSegmentMergeParallelIndexTaskRunner createPartialSegmentMergeRunner( - TaskToolbox toolbox, - List ioConfigs - ) - { - return new TestPartialSegmentMergeParallelIndexTaskRunner(toolbox, this, ioConfigs, getIndexingServiceClient()); - } - } - - private static class TestPartialSegmentGenerateRunner extends PartialSegmentGenerateParallelIndexTaskRunner - { - private final ParallelIndexSupervisorTask supervisorTask; - - private TestPartialSegmentGenerateRunner( - TaskToolbox toolbox, - ParallelIndexSupervisorTask supervisorTask, - IndexingServiceClient indexingServiceClient - ) - { - super( - toolbox, - supervisorTask.getId(), - supervisorTask.getGroupId(), - supervisorTask.getIngestionSchema(), - supervisorTask.getContext(), - indexingServiceClient - ); - this.supervisorTask = supervisorTask; - } - - @Override - Iterator> subTaskSpecIterator() throws IOException - { - final Iterator> iterator = super.subTaskSpecIterator(); - return new Iterator>() - { - @Override - public boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public SubTaskSpec next() - { - try { - Thread.sleep(10); - return iterator.next(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - }; - } - - @Override - SubTaskSpec newTaskSpec(InputSplit split) - { - final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( - getIngestionSchema().getDataSchema(), - new ParallelIndexIOConfig( - null, - getBaseInputSource().withSplit(split), - getIngestionSchema().getIOConfig().getInputFormat(), - getIngestionSchema().getIOConfig().isAppendToExisting() - ), - getIngestionSchema().getTuningConfig() - ); - return new SubTaskSpec( - getTaskId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), - getTaskId(), - getContext(), - split - ) - { - @Override - public PartialSegmentGenerateTask newSubTask(int numAttempts) - { - return new PartialSegmentGenerateTask( - null, - getGroupId(), - null, - getSupervisorTaskId(), - numAttempts, - subTaskIngestionSpec, - getContext(), - getIndexingServiceClient(), - new LocalParallelIndexTaskClientFactory(supervisorTask), - new TestAppenderatorsManager() - ); - } - }; - } - } - - private static class TestPartialSegmentMergeParallelIndexTaskRunner extends PartialSegmentMergeParallelIndexTaskRunner - { - private final ParallelIndexSupervisorTask supervisorTask; - - private TestPartialSegmentMergeParallelIndexTaskRunner( - TaskToolbox toolbox, - ParallelIndexSupervisorTask supervisorTask, - List mergeIOConfigs, - IndexingServiceClient indexingServiceClient - ) - { - super( - toolbox, - supervisorTask.getId(), - supervisorTask.getGroupId(), - supervisorTask.getIngestionSchema().getDataSchema(), - mergeIOConfigs, - supervisorTask.getIngestionSchema().getTuningConfig(), - supervisorTask.getContext(), - indexingServiceClient - ); - this.supervisorTask = supervisorTask; - } - - @Override - Iterator> subTaskSpecIterator() - { - final Iterator> iterator = super.subTaskSpecIterator(); - return new Iterator>() - { - @Override - public boolean hasNext() - { - return iterator.hasNext(); - } - - @Override - public SubTaskSpec next() - { - try { - Thread.sleep(10); - return iterator.next(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - }; - } - - @Override - SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) - { - final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec( - supervisorTask.getIngestionSchema().getDataSchema(), - ioConfig, - getTuningConfig() - ); - return new SubTaskSpec( - getTaskId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), - getTaskId(), - getContext(), - new InputSplit<>(ioConfig.getPartitionLocations()) - ) - { - @Override - public PartialSegmentMergeTask newSubTask(int numAttempts) - { - return new TestPartialSegmentMergeTask( - null, - getGroupId(), - null, - getSupervisorTaskId(), - numAttempts, - ingestionSpec, - getContext(), - getIndexingServiceClient(), - new LocalParallelIndexTaskClientFactory(supervisorTask), - getToolbox() - ); - } - }; - } - } - - private static class TestPartialSegmentMergeTask extends PartialSegmentMergeTask - { - private final TaskToolbox toolbox; - - private TestPartialSegmentMergeTask( - @Nullable String id, - String groupId, - TaskResource taskResource, - String supervisorTaskId, - int numAttempts, - PartialSegmentMergeIngestionSpec ingestionSchema, - Map context, - IndexingServiceClient indexingServiceClient, - IndexTaskClientFactory taskClientFactory, - TaskToolbox toolboxo - ) - { - super( - id, - groupId, - taskResource, - supervisorTaskId, - numAttempts, - ingestionSchema, - context, - indexingServiceClient, - taskClientFactory, - null - ); - this.toolbox = toolboxo; - } - - @Override - File fetchSegmentFile(File partitionDir, PartitionLocation location) - { - final File zippedFile = toolbox.getIntermediaryDataManager().findPartitionFile( - getSupervisorTaskId(), - location.getSubTaskId(), - location.getInterval(), - location.getPartitionId() - ); - if (zippedFile == null) { - throw new ISE("Can't find segment file for location[%s] at path[%s]", location); - } - return zippedFile; - } - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java index 23fdd2a9b4ab..70eb02404891 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java @@ -19,10 +19,10 @@ package org.apache.druid.indexing.common.task.batch.parallel.iterator; -import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowIterator; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.easymock.EasyMock; @@ -43,7 +43,7 @@ public class DefaultIndexTaskInputRowIteratorBuilderTest { public static class BuildTest { - private static final Firehose FIRHOSE = EasyMock.mock(Firehose.class); + private static final CloseableIterator ITERATOR = EasyMock.mock(CloseableIterator.class); private static final GranularitySpec GRANULARITY_SPEC = EasyMock.mock(GranularitySpec.class); private static final Runnable NULL_ROW_RUNNABLE = IndexTaskInputRowIteratorBuilder.NOOP_RUNNABLE; private static final Consumer ABSENT_BUCKET_INTERVAL_CONSUMER = @@ -53,10 +53,10 @@ public static class BuildTest public ExpectedException exception = ExpectedException.none(); @Test - public void requiresFirehose() + public void requiresDelegate() { exception.expect(NullPointerException.class); - exception.expectMessage("firehose required"); + exception.expectMessage("delegate required"); new DefaultIndexTaskInputRowIteratorBuilder() .granularitySpec(GRANULARITY_SPEC) @@ -72,7 +72,7 @@ public void requiresGranularitySpec() exception.expectMessage("granularitySpec required"); new DefaultIndexTaskInputRowIteratorBuilder() - .firehose(FIRHOSE) + .delegate(ITERATOR) .nullRowRunnable(NULL_ROW_RUNNABLE) .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) .build(); @@ -85,7 +85,7 @@ public void requiresNullRowHandler() exception.expectMessage("nullRowRunnable required"); new DefaultIndexTaskInputRowIteratorBuilder() - .firehose(FIRHOSE) + .delegate(ITERATOR) .granularitySpec(GRANULARITY_SPEC) .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) .build(); @@ -98,7 +98,7 @@ public void requiresAbsentBucketIntervalHandler() exception.expectMessage("absentBucketIntervalConsumer required"); new DefaultIndexTaskInputRowIteratorBuilder() - .firehose(FIRHOSE) + .delegate(ITERATOR) .granularitySpec(GRANULARITY_SPEC) .nullRowRunnable(NULL_ROW_RUNNABLE) .build(); @@ -108,7 +108,7 @@ public void requiresAbsentBucketIntervalHandler() public void succeedsIfAllRequiredPresent() { new DefaultIndexTaskInputRowIteratorBuilder() - .firehose(FIRHOSE) + .delegate(ITERATOR) .granularitySpec(GRANULARITY_SPEC) .nullRowRunnable(NULL_ROW_RUNNABLE) .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) @@ -129,12 +129,12 @@ public static class HandlerTest public void invokesNullRowHandlerFirst() { DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); - Firehose nullRowFirehose = Factory.createFirehose(null); + CloseableIterator nullInputRowIterator = Factory.createInputRowIterator(null); GranularitySpec absentBucketIntervalGranularitySpec = Factory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( - nullRowFirehose, + nullInputRowIterator, absentBucketIntervalGranularitySpec, NO_NEXT_INPUT_ROW ); @@ -147,14 +147,14 @@ public void invokesInvalidTimestampHandlerBeforeAbsentBucketIntervalHandler() { DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); InputRow inputRow = Factory.createInputRow(invalidTimestamp); - Firehose firehose = Factory.createFirehose(inputRow); + CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); GranularitySpec absentBucketIntervalGranularitySpec = Factory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); exception.expect(ParseException.class); exception.expectMessage("Encountered row with timestamp that cannot be represented as a long"); - HANDLER_TESTER.invokeHandlers(firehose, absentBucketIntervalGranularitySpec, NO_NEXT_INPUT_ROW); + HANDLER_TESTER.invokeHandlers(inputRowIterator, absentBucketIntervalGranularitySpec, NO_NEXT_INPUT_ROW); } @Test @@ -162,11 +162,11 @@ public void invokesAbsentBucketIntervalHandlerLast() { DateTime timestamp = Factory.TIMESTAMP; InputRow inputRow = Factory.createInputRow(timestamp); - Firehose firehose = Factory.createFirehose(inputRow); + CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); GranularitySpec absentBucketIntervalGranularitySpec = Factory.createAbsentBucketIntervalGranularitySpec(timestamp); List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( - firehose, + inputRowIterator, absentBucketIntervalGranularitySpec, NO_NEXT_INPUT_ROW ); @@ -182,13 +182,13 @@ public void invokesAppendedHandlersLast() { DateTime timestamp = Factory.TIMESTAMP; InputRow inputRow = Factory.createInputRow(timestamp); - Firehose firehose = Factory.createFirehose(inputRow); + CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); GranularitySpec granularitySpec = Factory.createGranularitySpec(timestamp, Factory.PRESENT_BUCKET_INTERVAL_OPT); - List appendedHandlers = Collections.singletonList(row -> true); + List appendedHandlers = Collections.singletonList(row -> true); List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( - firehose, + inputRowIterator, granularitySpec, appendedHandlers, NO_NEXT_INPUT_ROW @@ -205,11 +205,11 @@ public void doesNotInvokeHandlersIfRowValid() { DateTime timestamp = DateTimes.utc(0); InputRow inputRow = Factory.createInputRow(timestamp); - Firehose firehose = Factory.createFirehose(inputRow); + CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); GranularitySpec granularitySpec = Factory.createGranularitySpec(timestamp, Factory.PRESENT_BUCKET_INTERVAL_OPT); List handlerInvocationHistory = - HANDLER_TESTER.invokeHandlers(firehose, granularitySpec, inputRow); + HANDLER_TESTER.invokeHandlers(inputRowIterator, granularitySpec, inputRow); Assert.assertEquals(Collections.emptyList(), handlerInvocationHistory); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java index a9a146ba90a6..b2bf4fd51dc8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java @@ -20,19 +20,17 @@ package org.apache.druid.indexing.common.task.batch.parallel.iterator; import com.google.common.base.Optional; -import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.HandlingInputRowIterator; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowIterator; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; -import java.io.IOException; -import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -58,19 +56,28 @@ static InputRow createInputRow(DateTime timestamp, List dimensionValues) return inputRow; } - static Firehose createFirehose(InputRow inputRow) + static CloseableIterator createInputRowIterator(InputRow inputRow) { - Firehose firehose = EasyMock.mock(Firehose.class); - try { - EasyMock.expect(firehose.hasMore()).andStubReturn(true); - EasyMock.expect(firehose.nextRow()).andStubReturn(inputRow); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - EasyMock.replay(firehose); + return new CloseableIterator() + { + @Override + public void close() + { + // nothing + } - return firehose; + @Override + public boolean hasNext() + { + return true; + } + + @Override + public InputRow next() + { + return inputRow; + } + }; } static GranularitySpec createAbsentBucketIntervalGranularitySpec(DateTime timestamp) @@ -108,13 +115,13 @@ private HandlerTester(Supplier iteratorBuilder } List invokeHandlers( - Firehose firehose, + CloseableIterator inputRowIterator, GranularitySpec granularitySpec, InputRow expectedNextInputRow ) { return invokeHandlers( - firehose, + inputRowIterator, granularitySpec, Collections.emptyList(), expectedNextInputRow @@ -122,15 +129,15 @@ List invokeHandlers( } List invokeHandlers( - Firehose firehose, + CloseableIterator inputRowIterator, GranularitySpec granularitySpec, - List appendedHandlers, + List appendedHandlers, InputRow expectedNextInputRow ) { List handlerInvocationHistory = new ArrayList<>(); IndexTaskInputRowIteratorBuilder iteratorBuilder = iteratorBuilderSupplier.get() - .firehose(firehose) + .delegate(inputRowIterator) .granularitySpec(granularitySpec) .nullRowRunnable(() -> handlerInvocationHistory.add(Handler.NULL_ROW)) .absentBucketIntervalConsumer(row -> handlerInvocationHistory.add(Handler.ABSENT_BUCKET_INTERVAL)); @@ -141,7 +148,7 @@ List invokeHandlers( .forEach(((DefaultIndexTaskInputRowIteratorBuilder) iteratorBuilder)::appendInputRowHandler); } - InputRowIterator iterator = iteratorBuilder.build(); + HandlingInputRowIterator iterator = iteratorBuilder.build(); InputRow nextInputRow = iterator.next(); Assert.assertEquals(expectedNextInputRow, nextInputRow); From 1c3b6bf221643ad521b18894417d5da6a9ceb878 Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Mon, 18 Nov 2019 17:22:13 -0800 Subject: [PATCH 09/12] Address review comments --- .travis.yml | 3 -- .../common/task/AbstractBatchIndexTask.java | 6 +++ .../common/task/InputSourceProcessor.java | 4 +- ...utSourceSplitParallelIndexTaskRunner.java} | 9 ++-- .../parallel/ParallelIndexPhaseRunner.java | 41 +------------------ ...egmentGenerateParallelIndexTaskRunner.java | 2 +- .../parallel/PartialSegmentMergeTask.java | 4 +- .../batch/parallel/PartitionLocation.java | 4 +- .../task/batch/parallel/PartitionStat.java | 6 +++ .../parallel/PerfectRollupWorkerTask.java | 7 ---- ...faultIndexTaskInputRowIteratorBuilder.java | 5 ++- 11 files changed, 29 insertions(+), 62 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/{FirehoseSplitParallelIndexTaskRunner.java => InputSourceSplitParallelIndexTaskRunner.java} (93%) diff --git a/.travis.yml b/.travis.yml index 2ce508db7369..f4aa88b4bdcc 100644 --- a/.travis.yml +++ b/.travis.yml @@ -277,9 +277,6 @@ jobs: echo $v dmesg ======================== ; docker exec -it druid-$v sh -c 'dmesg | tail -3' ; done - - for v in ~/shared/tasklogs/*.log ; do - echo $v logtail ======================== ; tail -100 $v ; - done - &integration_kafka_index name: "kafka index integration test" diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index d32ddfb823a4..be857fcf1d45 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -188,6 +188,12 @@ public abstract List findSegmentsToLock(TaskActionClient taskAction @Nullable public abstract Granularity getSegmentGranularity(); + @Override + public final int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); + } + public boolean isUseSegmentLock() { return useSegmentLock; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index 5286e0729510..42779dafb1e0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -133,9 +133,11 @@ public SegmentsAndMetadata process( continue; } + // IndexTaskInputRowIteratorBuilder.absentBucketIntervalConsumer() ensures the interval will be present here Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); - @SuppressWarnings("OptionalGetWithoutIsPresent") // always present via IndexTaskInputRowIteratorBuilder + @SuppressWarnings("OptionalGetWithoutIsPresent") final Interval interval = optInterval.get(); + final String sequenceName = segmentAllocator.getSequenceName(interval, inputRow); final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java similarity index 93% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java index f58b96e0eb80..c526ba7b8647 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/FirehoseSplitParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java @@ -33,15 +33,15 @@ /** * Base class for different implementations of {@link ParallelIndexTaskRunner} that operate on - * {@link org.apache.druid.data.input.Firehose} splits. + * {@link org.apache.druid.data.input.InputSource} splits. */ -abstract class FirehoseSplitParallelIndexTaskRunner +abstract class InputSourceSplitParallelIndexTaskRunner extends ParallelIndexPhaseRunner { private final ParallelIndexIngestionSpec ingestionSchema; private final SplittableInputSource baseInputSource; - FirehoseSplitParallelIndexTaskRunner( + InputSourceSplitParallelIndexTaskRunner( TaskToolbox toolbox, String taskId, String groupId, @@ -115,6 +115,9 @@ final SubTaskSpec newTaskSpec(InputSplit split) ); } + /** + * @return Ingestion spec split suitable for this parallel worker + */ abstract SubTaskSpec createSubTaskSpec( String id, String groupId, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java index ca1986b8d077..ac7305e92d23 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java @@ -21,7 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.collect.ForwardingIterator; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -115,44 +114,6 @@ public abstract class ParallelIndexPhaseRunner> - { - private final Iterator> delegate; - - UniqueSubTaskIdDecorator(Iterator> subTaskSpecIterator) - { - delegate = subTaskSpecIterator; - } - - @Override - protected Iterator> delegate() - { - return delegate; - } - - @Override - public SubTaskSpec next() - { - ensureUniqueSubtaskId(); - return super.next(); - } - - private void ensureUniqueSubtaskId() - { - try { - // Ensure each subtask has a different id (which includes the current time -- see AbstractTask.getOrMakeId()) - Thread.sleep(1); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - } - @Override public TaskState run() throws Exception { @@ -161,7 +122,7 @@ public TaskState run() throws Exception return TaskState.SUCCESS; } - final Iterator> subTaskSpecIterator = new UniqueSubTaskIdDecorator(subTaskSpecIterator()); + final Iterator> subTaskSpecIterator = subTaskSpecIterator(); final long taskStatusCheckingPeriod = tuningConfig.getTaskStatusCheckPeriodMs(); taskMonitor = new TaskMonitor<>( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java index 50bef2ebcd11..db92233aab30 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java @@ -34,7 +34,7 @@ * @see PartialHashSegmentMergeParallelIndexTaskRunner */ class PartialHashSegmentGenerateParallelIndexTaskRunner - extends FirehoseSplitParallelIndexTaskRunner + extends InputSourceSplitParallelIndexTaskRunner { // For tests private final IndexTaskClientFactory taskClientFactory; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 30b7a57bb9a7..ea0ac936925c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -93,8 +92,7 @@ abstract class PartialSegmentMergeTask private final Interval interval; private final T secondaryPartition; - @JsonCreator - public PartitionLocation( + PartitionLocation( String host, int port, boolean useHttps, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java index 79c59eda0bc8..66974c297eea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java @@ -101,8 +101,14 @@ public final Long getSizeBytes() return sizeBytes; } + /** + * @return Uniquely identifying index from 0..N-1 of the N partitions + */ abstract int getPartitionId(); + /** + * @return Definition of secondary partition. For example, for range partitioning, this should include the start/end. + */ abstract T getSecondaryPartition(); @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java index 82b8211b193b..7cce1d32e833 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java @@ -24,7 +24,6 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; @@ -82,12 +81,6 @@ private static void checkPartitionsSpec(PartitionsSpec partitionsSpec) } } - @Override - public final int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); - } - @Override public final boolean requireLockExistingSegments() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java index b03bbd4d466f..3a8ad8ab566c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java @@ -49,6 +49,9 @@ * {@link GranularitySpec} bucket intervals, invoke the absent bucket interval {@link Consumer} * callback. * + * 4. Any additional handlers in the order they are added by calls to + * {@link #appendInputRowHandler(HandlingInputRowIterator.InputRowHandler)}. + * * If any of the handlers invoke their respective callback, the {@link HandlingInputRowIterator} will yield * a null {@link InputRow} next; otherwise, the next {@link InputRow} is yielded. * @@ -59,7 +62,7 @@ public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRo private GranularitySpec granularitySpec = null; private HandlingInputRowIterator.InputRowHandler nullRowHandler = null; private HandlingInputRowIterator.InputRowHandler absentBucketIntervalHandler = null; - private List appendedInputRowHandlers = new ArrayList<>(); + private final List appendedInputRowHandlers = new ArrayList<>(); @Override public DefaultIndexTaskInputRowIteratorBuilder delegate(CloseableIterator inputRowIterator) From cb9b6c6af18cb67e361f6a636d855f6fdde2d8cf Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Mon, 18 Nov 2019 21:30:46 -0800 Subject: [PATCH 10/12] Address more review comments --- .../data/input/HandlingInputRowIterator.java | 12 +- .../common/task/AbstractBatchIndexTask.java | 2 +- .../indexing/common/task/HadoopIndexTask.java | 6 - .../druid/indexing/common/task/IndexTask.java | 6 - .../common/task/InputSourceProcessor.java | 17 +- .../parallel/ParallelIndexSupervisorTask.java | 6 - .../batch/parallel/SinglePhaseSubTask.java | 6 - ...faultCachingLocalSegmentAllocatorTest.java | 145 ++++++++++++++++++ 8 files changed, 165 insertions(+), 35 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java diff --git a/core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java b/core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java index 238ba3d69c8b..fa2c6d7235c0 100644 --- a/core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java +++ b/core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java @@ -22,13 +22,13 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import javax.annotation.Nullable; -import java.util.Iterator; +import java.io.IOException; import java.util.List; /** * Decorated {@link CloseableIterator} that can process rows with {@link InputRowHandler}s. */ -public class HandlingInputRowIterator implements Iterator +public class HandlingInputRowIterator implements CloseableIterator { @FunctionalInterface public interface InputRowHandler @@ -72,6 +72,8 @@ public InputRow next() { InputRow inputRow = delegate.next(); + // NOTE: This loop invokes a virtual call per input row, which may have significant overhead for large inputs + // (e.g. InputSourceProcessor). If performance suffers, this implementation or the clients will need to change. for (InputRowHandler inputRowHandler : inputRowHandlers) { if (inputRowHandler.handle(inputRow)) { return null; @@ -80,5 +82,11 @@ public InputRow next() return inputRow; } + + @Override + public void close() throws IOException + { + delegate.close(); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index be857fcf1d45..bcb799b966a5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -189,7 +189,7 @@ public abstract List findSegmentsToLock(TaskActionClient taskAction public abstract Granularity getSegmentGranularity(); @Override - public final int getPriority() + public int getPriority() { return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 57081f52afb6..a4df10e53ed4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -179,12 +179,6 @@ public HadoopIndexTask( this.ingestionState = IngestionState.NOT_STARTED; } - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); - } - @Override public String getType() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index f5b84f2906df..75a30663b47f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -254,12 +254,6 @@ public IndexTask( this.appenderatorsManager = appenderatorsManager; } - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); - } - @Override public String getType() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java index 42779dafb1e0..e28491ef3032 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java @@ -118,14 +118,15 @@ public SegmentsAndMetadata process( tmpDir ) ); - try (final CloseableIterator inputRowIterator = inputSourceReader.read()) { - HandlingInputRowIterator iterator = inputRowIteratorBuilder - .delegate(inputRowIterator) - .granularitySpec(granularitySpec) - .nullRowRunnable(buildSegmentsMeters::incrementThrownAway) - .absentBucketIntervalConsumer(inputRow -> buildSegmentsMeters.incrementThrownAway()) - .build(); - + try ( + final CloseableIterator inputRowIterator = inputSourceReader.read(); + HandlingInputRowIterator iterator = inputRowIteratorBuilder + .delegate(inputRowIterator) + .granularitySpec(granularitySpec) + .nullRowRunnable(buildSegmentsMeters::incrementThrownAway) + .absentBucketIntervalConsumer(inputRow -> buildSegmentsMeters.incrementThrownAway()) + .build() + ) { while (iterator.hasNext()) { try { final InputRow inputRow = iterator.next(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 8c277ea16299..f8300d4671f7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -219,12 +219,6 @@ private static void checkPartitionsSpecForForceGuaranteedRollup(PartitionsSpec p } } - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); - } - @Override public String getType() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 6408299d084f..da6f070fb4a7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -167,12 +167,6 @@ public SinglePhaseSubTask( } } - @Override - public int getPriority() - { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); - } - @Override public String getType() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java new file mode 100644 index 000000000000..181067784a1e --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.task.DefaultCachingLocalSegmentAllocator; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class DefaultCachingLocalSegmentAllocatorTest +{ + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String DATASOURCE = "datasource"; + private static final String TASKID = "taskid"; + private static final Interval INTERVAL = Intervals.utc(0, 1000); + private static final String VERSION = "version"; + private static final String DIMENSION = "dim"; + private static final List PARTITION_DIMENSIONS = ImmutableList.of(DIMENSION); + private static final int NUM_PARTITONS = 1; + private static final ShardSpecFactory SHARD_SPEC_FACTORY = new HashBasedNumberedShardSpecFactory( + PARTITION_DIMENSIONS, + NUM_PARTITONS + ); + private static final int PARTITION_NUM = 0; + private static final Map> ALLOCATE_SPEC = ImmutableMap.of( + INTERVAL, Pair.of(SHARD_SPEC_FACTORY, NUM_PARTITONS) + ); + + private DefaultCachingLocalSegmentAllocator target; + + @Before + public void setup() throws IOException + { + TaskToolbox toolbox = createToolbox(); + target = new DefaultCachingLocalSegmentAllocator( + toolbox, + TASKID, + DATASOURCE, + ALLOCATE_SPEC + ); + } + + @Test + public void allocatesCorrectShardSpec() throws IOException + { + InputRow row = createInputRow(); + + String sequenceName = target.getSequenceName(INTERVAL, row); + SegmentIdWithShardSpec segmentIdWithShardSpec = target.allocate(row, sequenceName, null, false); + + Assert.assertEquals( + SegmentId.of(DATASOURCE, INTERVAL, VERSION, PARTITION_NUM), + segmentIdWithShardSpec.asSegmentId() + ); + HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segmentIdWithShardSpec.getShardSpec(); + Assert.assertEquals(PARTITION_DIMENSIONS, shardSpec.getPartitionDimensions()); + Assert.assertEquals(NUM_PARTITONS, shardSpec.getPartitions()); + Assert.assertEquals(PARTITION_NUM, shardSpec.getPartitionNum()); + } + + private static TaskToolbox createToolbox() + { + TaskToolbox toolbox = EasyMock.mock(TaskToolbox.class); + EasyMock.expect(toolbox.getTaskActionClient()).andStubReturn(createTaskActionClient()); + EasyMock.expect(toolbox.getObjectMapper()).andStubReturn(OBJECT_MAPPER); + EasyMock.replay(toolbox); + return toolbox; + } + + private static TaskActionClient createTaskActionClient() + { + List taskLocks = Collections.singletonList(createTaskLock()); + + try { + TaskActionClient taskActionClient = EasyMock.mock(TaskActionClient.class); + EasyMock.expect(taskActionClient.submit(EasyMock.anyObject(LockListAction.class))).andStubReturn(taskLocks); + EasyMock.replay(taskActionClient); + return taskActionClient; + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static TaskLock createTaskLock() + { + TaskLock taskLock = EasyMock.mock(TaskLock.class); + EasyMock.expect(taskLock.getInterval()).andStubReturn(INTERVAL); + EasyMock.expect(taskLock.getVersion()).andStubReturn(VERSION); + EasyMock.replay(taskLock); + return taskLock; + } + + private static InputRow createInputRow() + { + long timestamp = INTERVAL.getStartMillis(); + InputRow inputRow = EasyMock.mock(InputRow.class); + EasyMock.expect(inputRow.getTimestamp()).andStubReturn(new DateTime(timestamp)); + EasyMock.expect(inputRow.getTimestampFromEpoch()).andStubReturn(timestamp); + EasyMock.expect(inputRow.getDimension(DIMENSION)).andStubReturn(Collections.singletonList(DIMENSION)); + EasyMock.replay(inputRow); + return inputRow; + } +} From 09816361feea2c9bc9e45949a12586a5b828e7a5 Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Tue, 19 Nov 2019 09:53:10 -0800 Subject: [PATCH 11/12] Fix forbidden apis --- .../parallel/DefaultCachingLocalSegmentAllocatorTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java index 181067784a1e..549510c61353 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.DefaultCachingLocalSegmentAllocator; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -36,7 +37,6 @@ import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.ShardSpecFactory; import org.easymock.EasyMock; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -136,7 +136,7 @@ private static InputRow createInputRow() { long timestamp = INTERVAL.getStartMillis(); InputRow inputRow = EasyMock.mock(InputRow.class); - EasyMock.expect(inputRow.getTimestamp()).andStubReturn(new DateTime(timestamp)); + EasyMock.expect(inputRow.getTimestamp()).andStubReturn(DateTimes.utc(timestamp)); EasyMock.expect(inputRow.getTimestampFromEpoch()).andStubReturn(timestamp); EasyMock.expect(inputRow.getDimension(DIMENSION)).andStubReturn(Collections.singletonList(DIMENSION)); EasyMock.replay(inputRow); From 43f046a309b8123ce0575c70d94e07b2b1fde0c5 Mon Sep 17 00:00:00 2001 From: Chi Cao Minh Date: Wed, 20 Nov 2019 14:58:03 -0800 Subject: [PATCH 12/12] Address more review comments --- .../input/HandlingInputRowIteratorTest.java | 78 +++++++------- .../task/CachingLocalSegmentAllocator.java | 2 +- ...artitionCachingLocalSegmentAllocator.java} | 6 +- .../druid/indexing/common/task/IndexTask.java | 2 +- .../PartialHashSegmentGenerateTask.java | 4 +- .../druid/indexing/common/TestUtils.java | 56 +++++++--- .../task/NoopIndexTaskClientFactory.java | 4 +- .../GeneratedHashPartitionsReportTest.java | 16 +-- ...tionCachingLocalSegmentAllocatorTest.java} | 8 +- .../batch/parallel/HashPartitionStatTest.java | 16 +-- ....java => ParallelIndexTestingFactory.java} | 88 ++++++--------- .../PartialHashSegmentGenerateTaskTest.java | 34 +++--- .../PartialHashSegmentMergeIOConfigTest.java | 14 +-- ...tialHashSegmentMergeIngestionSpecTest.java | 18 ++-- .../PartialHashSegmentMergeTaskTest.java | 36 +++---- ...tIndexTaskInputRowIteratorBuilderTest.java | 100 +++++++++++------- ...nputRowIteratorBuilderTestingFactory.java} | 5 +- 17 files changed, 259 insertions(+), 228 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{DefaultCachingLocalSegmentAllocator.java => HashPartitionCachingLocalSegmentAllocator.java} (93%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/{DefaultCachingLocalSegmentAllocatorTest.java => HashPartitionCachingLocalSegmentAllocatorTest.java} (95%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/{Factory.java => ParallelIndexTestingFactory.java} (72%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/{Factory.java => IndexTaskInputRowIteratorBuilderTestingFactory.java} (97%) diff --git a/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java b/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java index 88ddd5c0dff4..8c907badabf1 100644 --- a/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java +++ b/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.data.input; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.easymock.EasyMock; import org.junit.Assert; @@ -39,21 +40,23 @@ public class HandlingInputRowIteratorTest { public static class AbsentRowTest { - private static final CloseableIterator EMPTY_ITERATOR = new TestCloseableIterator() - { - @Override - public boolean hasNext() - { - return false; - } - - @Nullable - @Override - public InputRow next() - { - throw new NoSuchElementException(); - } - }; + private static final CloseableIterator EMPTY_ITERATOR = CloseableIterators.withEmptyBaggage( + new Iterator() + { + @Override + public boolean hasNext() + { + return false; + } + + @Nullable + @Override + public InputRow next() + { + throw new NoSuchElementException(); + } + } + ); private HandlingInputRowIterator target; @@ -131,23 +134,25 @@ private static HandlingInputRowIterator createInputRowIterator( HandlingInputRowIterator.InputRowHandler secondHandler ) { - CloseableIterator iterator = new TestCloseableIterator() - { - private final Iterator delegate = INPUT_ROWS.iterator(); - - @Override - public boolean hasNext() - { - return delegate.hasNext(); - } - - @Nullable - @Override - public InputRow next() - { - return delegate.next(); - } - }; + CloseableIterator iterator = CloseableIterators.withEmptyBaggage( + new Iterator() + { + private final Iterator delegate = INPUT_ROWS.iterator(); + + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + @Nullable + @Override + public InputRow next() + { + return delegate.next(); + } + } + ); return new HandlingInputRowIterator(iterator, Arrays.asList(firstHandler, secondHandler)); } @@ -171,13 +176,4 @@ public boolean handle(InputRow inputRow) } } } - - private abstract static class TestCloseableIterator implements CloseableIterator - { - @Override - public void close() - { - throw new UnsupportedOperationException(); - } - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index 4642ff336d12..21157bf13957 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java @@ -42,7 +42,7 @@ /** * Allocates all necessary segments locally at the beginning and reuses them. * - * @see DefaultCachingLocalSegmentAllocator + * @see HashPartitionCachingLocalSegmentAllocator */ class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/DefaultCachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HashPartitionCachingLocalSegmentAllocator.java similarity index 93% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/DefaultCachingLocalSegmentAllocator.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/HashPartitionCachingLocalSegmentAllocator.java index 84ef1e396b01..9640ed461358 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/DefaultCachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HashPartitionCachingLocalSegmentAllocator.java @@ -37,18 +37,18 @@ import java.util.stream.IntStream; /** - * Allocates all necessary segments locally at the beginning and reuses them. + * Allocates all necessary hash-partitioned segments locally at the beginning and reuses them. * * @see CachingLocalSegmentAllocator */ -public class DefaultCachingLocalSegmentAllocator implements IndexTaskSegmentAllocator +public class HashPartitionCachingLocalSegmentAllocator implements IndexTaskSegmentAllocator { private final TaskToolbox toolbox; private final String dataSource; private final Map> allocateSpec; private final IndexTaskSegmentAllocator delegate; - public DefaultCachingLocalSegmentAllocator( + public HashPartitionCachingLocalSegmentAllocator( TaskToolbox toolbox, String taskId, String dataSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 73fcb6d03cfb..501c2cd8957d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -827,7 +827,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( // We use the timeChunk lock and don't have to ask the overlord to create segmentIds. // Instead, a local allocator is used. if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { - return new DefaultCachingLocalSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); + return new HashPartitionCachingLocalSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); } else { return new LocalSegmentAllocator(toolbox, getId(), getDataSource(), dataSchema.getGranularitySpec()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index 9f7f4d65b5a1..d7f886207719 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -26,7 +26,7 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.DefaultCachingLocalSegmentAllocator; +import org.apache.druid.indexing.common.task.HashPartitionCachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; import org.apache.druid.indexing.common.task.TaskResource; @@ -127,7 +127,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception @Override IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException { - return new DefaultCachingLocalSegmentAllocator( + return new HashPartitionCachingLocalSegmentAllocator( toolbox, getId(), getDataSource(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java index 723722343e60..15f59189bc27 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestUtils.java @@ -20,28 +20,34 @@ package org.apache.druid.indexing.common; import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.data.input.impl.NoopInputFormat; import org.apache.druid.data.input.impl.NoopInputSource; +import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.NoopIndexTaskClientFactory; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; @@ -51,13 +57,42 @@ import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; +import org.joda.time.Duration; +import java.util.List; import java.util.concurrent.TimeUnit; /** + * */ public class TestUtils { + public static final IndexingServiceClient INDEXING_SERVICE_CLIENT = new NoopIndexingServiceClient(); + public static final IndexTaskClientFactory TASK_CLIENT_FACTORY = new NoopIndexTaskClientFactory<>(); + public static final AppenderatorsManager APPENDERATORS_MANAGER = new TestAppenderatorsManager(); + + public static final HttpClient SHUFFLE_CLIENT = new HttpClient() + { + @Override + public ListenableFuture go( + Request request, + HttpResponseHandler handler + ) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture go( + Request request, + HttpResponseHandler handler, + Duration readTimeout + ) + { + throw new UnsupportedOperationException(); + } + }; + private static final Logger log = new Logger(TestUtils.class); private final ObjectMapper jsonMapper; @@ -70,14 +105,7 @@ public TestUtils() this.jsonMapper = new DefaultObjectMapper(); indexIO = new IndexIO( jsonMapper, - new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 0; - } - } + () -> 0 ); indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); @@ -93,11 +121,12 @@ public int columnCacheSizeBytes() .addValue(AuthorizerMapper.class, null) .addValue(RowIngestionMetersFactory.class, rowIngestionMetersFactory) .addValue(PruneSpecsHolder.class, PruneSpecsHolder.DEFAULT) - .addValue(IndexingServiceClient.class, new NoopIndexingServiceClient()) + .addValue(IndexingServiceClient.class, INDEXING_SERVICE_CLIENT) .addValue(AuthorizerMapper.class, new AuthorizerMapper(ImmutableMap.of())) - .addValue(AppenderatorsManager.class, new TestAppenderatorsManager()) + .addValue(AppenderatorsManager.class, APPENDERATORS_MANAGER) .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) - .addValue(IndexTaskClientFactory.class, new NoopIndexTaskClientFactory()) + .addValue(IndexTaskClientFactory.class, TASK_CLIENT_FACTORY) + .addValue(HttpClient.class, SHUFFLE_CLIENT) ); jsonMapper.registerModule( @@ -114,6 +143,9 @@ public void setupModule(SetupContext context) } } ); + + List firehoseModules = new FirehoseModule().getJacksonModules(); + firehoseModules.forEach(jsonMapper::registerModule); } public ObjectMapper getTestObjectMapper() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java index 8138f7c5c718..8296d21a9634 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NoopIndexTaskClientFactory.java @@ -23,10 +23,10 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.joda.time.Duration; -public class NoopIndexTaskClientFactory implements IndexTaskClientFactory +public class NoopIndexTaskClientFactory implements IndexTaskClientFactory { @Override - public IndexTaskClient build( + public T build( TaskInfoProvider taskInfoProvider, String callerId, int numThreads, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java index 6e4bcd2be383..1343b9425833 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java @@ -28,7 +28,7 @@ public class GeneratedHashPartitionsReportTest { - private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); private GeneratedHashPartitionsReport target; @@ -39,13 +39,13 @@ public void setup() "task-id", Collections.singletonList( new HashPartitionStat( - Factory.TASK_EXECUTOR_HOST, - Factory.TASK_EXECUTOR_PORT, - Factory.USE_HTTPS, - Factory.INTERVAL, - Factory.PARTITION_ID, - Factory.NUM_ROWS, - Factory.SIZE_BYTES + ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, + ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.PARTITION_ID, + ParallelIndexTestingFactory.NUM_ROWS, + ParallelIndexTestingFactory.SIZE_BYTES ) ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java similarity index 95% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index eef80e9381a4..5b60bdf7a610 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/DefaultCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -27,7 +27,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.DefaultCachingLocalSegmentAllocator; +import org.apache.druid.indexing.common.task.HashPartitionCachingLocalSegmentAllocator; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -48,7 +48,7 @@ import java.util.List; import java.util.Map; -public class DefaultCachingLocalSegmentAllocatorTest +public class HashPartitionCachingLocalSegmentAllocatorTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final String DATASOURCE = "datasource"; @@ -67,13 +67,13 @@ public class DefaultCachingLocalSegmentAllocatorTest INTERVAL, Pair.of(SHARD_SPEC_FACTORY, NUM_PARTITONS) ); - private DefaultCachingLocalSegmentAllocator target; + private HashPartitionCachingLocalSegmentAllocator target; @Before public void setup() throws IOException { TaskToolbox toolbox = createToolbox(); - target = new DefaultCachingLocalSegmentAllocator( + target = new HashPartitionCachingLocalSegmentAllocator( toolbox, TASKID, DATASOURCE, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java index 9a5caf2a8968..1eb6f867cd8e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java @@ -27,7 +27,7 @@ public class HashPartitionStatTest { - private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); private HashPartitionStat target; @@ -35,13 +35,13 @@ public class HashPartitionStatTest public void setup() { target = new HashPartitionStat( - Factory.TASK_EXECUTOR_HOST, - Factory.TASK_EXECUTOR_PORT, - Factory.USE_HTTPS, - Factory.INTERVAL, - Factory.PARTITION_ID, - Factory.NUM_ROWS, - Factory.SIZE_BYTES + ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, + ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.PARTITION_ID, + ParallelIndexTestingFactory.NUM_ROWS, + ParallelIndexTestingFactory.SIZE_BYTES ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/Factory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java similarity index 72% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/Factory.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 3c57d02c4abe..3d6e86aa01f2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/Factory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -20,35 +20,30 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; +import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.jackson.JacksonModule; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.segment.TestHelper; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory; +import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -import org.easymock.EasyMock; import org.joda.time.Duration; import org.joda.time.Interval; @@ -57,7 +52,10 @@ import java.util.List; import java.util.Map; -class Factory +/** + * Helper for creating objects for testing parallel indexing. + */ +class ParallelIndexTestingFactory { static final String AUTOMATIC_ID = null; static final String ID = "id"; @@ -66,10 +64,11 @@ class Factory static final String SUPERVISOR_TASK_ID = "supervisor-task-id"; static final int NUM_ATTEMPTS = 1; static final Map CONTEXT = Collections.emptyMap(); - static final IndexingServiceClient INDEXING_SERVICE_CLIENT = null; - static final IndexTaskClientFactory TASK_CLIENT_FACTORY = null; - static final AppenderatorsManager APPENDERATORS_MANAGER = null; - static final HttpClient SHUFFLE_CLIENT = null; + static final IndexingServiceClient INDEXING_SERVICE_CLIENT = TestUtils.INDEXING_SERVICE_CLIENT; + static final IndexTaskClientFactory TASK_CLIENT_FACTORY = + TestUtils.TASK_CLIENT_FACTORY; + static final AppenderatorsManager APPENDERATORS_MANAGER = TestUtils.APPENDERATORS_MANAGER; + static final HttpClient SHUFFLE_CLIENT = TestUtils.SHUFFLE_CLIENT; static final List INPUT_INTERVALS = Collections.singletonList(Intervals.ETERNITY); static final String TASK_EXECUTOR_HOST = "task-executor-host"; static final int TASK_EXECUTOR_PORT = 1; @@ -81,7 +80,8 @@ class Factory static final String HOST = "host"; static final int PORT = 1; static final String SUBTASK_ID = "subtask-id"; - private static final ObjectMapper NESTED_OBJECT_MAPPER = TestHelper.makeJsonMapper(); + private static final TestUtils TEST_UTILS = new TestUtils(); + private static final ObjectMapper NESTED_OBJECT_MAPPER = TEST_UTILS.getTestObjectMapper(); private static final String SCHEMA_TIME = "time"; private static final String SCHEMA_DIMENSION = "dim"; private static final String DATASOURCE = "datasource"; @@ -90,24 +90,12 @@ class Factory PARTITION_ID, PARTITION_ID + 1, Collections.singletonList("dim"), - Factory.NESTED_OBJECT_MAPPER + ParallelIndexTestingFactory.NESTED_OBJECT_MAPPER ); static ObjectMapper createObjectMapper() { - InjectableValues injectableValues = new InjectableValues.Std() - .addValue(IndexingServiceClient.class, INDEXING_SERVICE_CLIENT) - .addValue(IndexTaskClientFactory.class, TASK_CLIENT_FACTORY) - .addValue(AppenderatorsManager.class, APPENDERATORS_MANAGER) - .addValue(ObjectMapper.class, NESTED_OBJECT_MAPPER) - .addValue(HttpClient.class, SHUFFLE_CLIENT); - - ObjectMapper objectMapper = new JacksonModule().jsonMapper().setInjectableValues(injectableValues); - - List firehoseModule = new FirehoseModule().getJacksonModules(); - firehoseModule.forEach(objectMapper::registerModule); - - return objectMapper; + return TEST_UTILS.getTestObjectMapper(); } @SuppressWarnings("SameParameterValue") @@ -179,40 +167,33 @@ ParallelIndexTuningConfig build() static DataSchema createDataSchema(List granularitySpecInputIntervals) { GranularitySpec granularitySpec = new ArbitraryGranularitySpec(Granularities.DAY, granularitySpecInputIntervals); - - Map parser = NESTED_OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec(SCHEMA_TIME, "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION)), - null, - null - ), - null, - null - ), - null - ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + TimestampSpec timestampSpec = new TimestampSpec(SCHEMA_TIME, "auto", null); + DimensionsSpec dimensionsSpec = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION)), + null, + null ); return new DataSchema( DATASOURCE, - parser, - null, + timestampSpec, + dimensionsSpec, + new AggregatorFactory[]{}, granularitySpec, + TransformSpec.NONE, null, NESTED_OBJECT_MAPPER ); } static ParallelIndexIngestionSpec createIngestionSpec( - InlineFirehoseFactory inlineFirehoseFactory, + InputSource inputSource, + InputFormat inputFormat, ParallelIndexTuningConfig tuningConfig, DataSchema dataSchema ) { - ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(inlineFirehoseFactory, false); + ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(null, inputSource, inputFormat, false); return new ParallelIndexIngestionSpec(dataSchema, ioConfig, tuningConfig); } @@ -248,14 +229,7 @@ SingleDimensionPartitionsSpec build() static IndexTaskClientFactory createTaskClientFactory() { - return (taskInfoProvider, callerId, numThreads, httpTimeout, numRetries) -> createTaskClient(); - } - - private static ParallelIndexSupervisorTaskClient createTaskClient() - { - ParallelIndexSupervisorTaskClient taskClient = EasyMock.niceMock(ParallelIndexSupervisorTaskClient.class); - EasyMock.replay(taskClient); - return taskClient; + return TASK_CLIENT_FACTORY; } static String createRow(long timestamp, Object dimensionValue) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java index e7f55a35c27c..81a1f06b1032 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java @@ -20,20 +20,24 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory; import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.File; + public class PartialHashSegmentGenerateTaskTest { - private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); - private static final ParallelIndexIngestionSpec INGESTION_SPEC = Factory.createIngestionSpec( - new InlineFirehoseFactory("data"), - new Factory.TuningConfigBuilder().build(), - Factory.createDataSchema(Factory.INPUT_INTERVALS) + private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); + private static final ParallelIndexIngestionSpec INGESTION_SPEC = ParallelIndexTestingFactory.createIngestionSpec( + new LocalInputSource(new File("baseDir"), "filer"), + new JsonInputFormat(null, null), + new ParallelIndexTestingFactory.TuningConfigBuilder().build(), + ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS) ); private PartialHashSegmentGenerateTask target; @@ -42,16 +46,16 @@ public class PartialHashSegmentGenerateTaskTest public void setup() { target = new PartialHashSegmentGenerateTask( - Factory.AUTOMATIC_ID, - Factory.GROUP_ID, - Factory.TASK_RESOURCE, - Factory.SUPERVISOR_TASK_ID, - Factory.NUM_ATTEMPTS, + ParallelIndexTestingFactory.AUTOMATIC_ID, + ParallelIndexTestingFactory.GROUP_ID, + ParallelIndexTestingFactory.TASK_RESOURCE, + ParallelIndexTestingFactory.SUPERVISOR_TASK_ID, + ParallelIndexTestingFactory.NUM_ATTEMPTS, INGESTION_SPEC, - Factory.CONTEXT, - Factory.INDEXING_SERVICE_CLIENT, - Factory.TASK_CLIENT_FACTORY, - Factory.APPENDERATORS_MANAGER + ParallelIndexTestingFactory.CONTEXT, + ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT, + ParallelIndexTestingFactory.TASK_CLIENT_FACTORY, + ParallelIndexTestingFactory.APPENDERATORS_MANAGER ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java index b16c77eb2354..413c34d9d668 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java @@ -28,14 +28,14 @@ public class PartialHashSegmentMergeIOConfigTest { - private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - Factory.HOST, - Factory.PORT, - Factory.USE_HTTPS, - Factory.SUBTASK_ID, - Factory.INTERVAL, - Factory.PARTITION_ID + ParallelIndexTestingFactory.HOST, + ParallelIndexTestingFactory.PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.SUBTASK_ID, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.PARTITION_ID ); private PartialHashSegmentMergeIOConfig target; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java index e82f45fe26b8..d734739a0146 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java @@ -29,14 +29,14 @@ public class PartialHashSegmentMergeIngestionSpecTest { - private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - Factory.HOST, - Factory.PORT, - Factory.USE_HTTPS, - Factory.SUBTASK_ID, - Factory.INTERVAL, - Factory.PARTITION_ID + ParallelIndexTestingFactory.HOST, + ParallelIndexTestingFactory.PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.SUBTASK_ID, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.PARTITION_ID ); private static final PartialHashSegmentMergeIOConfig IO_CONFIG = new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); @@ -52,9 +52,9 @@ public class PartialHashSegmentMergeIngestionSpecTest public void setup() { target = new PartialHashSegmentMergeIngestionSpec( - Factory.createDataSchema(Factory.INPUT_INTERVALS), + ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), IO_CONFIG, - new Factory.TuningConfigBuilder() + new ParallelIndexTestingFactory.TuningConfigBuilder() .partitionsSpec(PARTITIONS_SPEC) .build() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java index 324befa3bce7..8a733a2fd223 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java @@ -31,14 +31,14 @@ public class PartialHashSegmentMergeTaskTest { - private static final ObjectMapper OBJECT_MAPPER = Factory.createObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - Factory.HOST, - Factory.PORT, - Factory.USE_HTTPS, - Factory.SUBTASK_ID, - Factory.INTERVAL, - Factory.PARTITION_ID + ParallelIndexTestingFactory.HOST, + ParallelIndexTestingFactory.PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.SUBTASK_ID, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.PARTITION_ID ); private static final PartialHashSegmentMergeIOConfig IO_CONFIG = new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); @@ -49,9 +49,9 @@ public class PartialHashSegmentMergeTaskTest ); private static final PartialHashSegmentMergeIngestionSpec INGESTION_SPEC = new PartialHashSegmentMergeIngestionSpec( - Factory.createDataSchema(Factory.INPUT_INTERVALS), + ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), IO_CONFIG, - new Factory.TuningConfigBuilder() + new ParallelIndexTestingFactory.TuningConfigBuilder() .partitionsSpec(PARTITIONS_SPEC) .build() ); @@ -62,16 +62,16 @@ public class PartialHashSegmentMergeTaskTest public void setup() { target = new PartialHashSegmentMergeTask( - Factory.AUTOMATIC_ID, - Factory.GROUP_ID, - Factory.TASK_RESOURCE, - Factory.SUPERVISOR_TASK_ID, - Factory.NUM_ATTEMPTS, + ParallelIndexTestingFactory.AUTOMATIC_ID, + ParallelIndexTestingFactory.GROUP_ID, + ParallelIndexTestingFactory.TASK_RESOURCE, + ParallelIndexTestingFactory.SUPERVISOR_TASK_ID, + ParallelIndexTestingFactory.NUM_ATTEMPTS, INGESTION_SPEC, - Factory.CONTEXT, - Factory.INDEXING_SERVICE_CLIENT, - Factory.TASK_CLIENT_FACTORY, - Factory.SHUFFLE_CLIENT + ParallelIndexTestingFactory.CONTEXT, + ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT, + ParallelIndexTestingFactory.TASK_CLIENT_FACTORY, + ParallelIndexTestingFactory.SHUFFLE_CLIENT ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java index 70eb02404891..c5dd5edfa77b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java @@ -118,8 +118,10 @@ public void succeedsIfAllRequiredPresent() public static class HandlerTest { - private static final Factory.HandlerTester HANDLER_TESTER = - Factory.createHandlerTester(DefaultIndexTaskInputRowIteratorBuilder::new); + private static final IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester HANDLER_TESTER = + IndexTaskInputRowIteratorBuilderTestingFactory.createHandlerTester( + DefaultIndexTaskInputRowIteratorBuilder::new + ); private static final InputRow NO_NEXT_INPUT_ROW = null; @Rule @@ -129,27 +131,33 @@ public static class HandlerTest public void invokesNullRowHandlerFirst() { DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); - CloseableIterator nullInputRowIterator = Factory.createInputRowIterator(null); + CloseableIterator nullInputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(null); GranularitySpec absentBucketIntervalGranularitySpec = - Factory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); + IndexTaskInputRowIteratorBuilderTestingFactory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); - List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( - nullInputRowIterator, - absentBucketIntervalGranularitySpec, - NO_NEXT_INPUT_ROW - ); + List handlerInvocationHistory = + HANDLER_TESTER.invokeHandlers( + nullInputRowIterator, + absentBucketIntervalGranularitySpec, + NO_NEXT_INPUT_ROW + ); - Assert.assertEquals(Collections.singletonList(Factory.HandlerTester.Handler.NULL_ROW), handlerInvocationHistory); + Assert.assertEquals( + Collections.singletonList(IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.NULL_ROW), + handlerInvocationHistory + ); } @Test public void invokesInvalidTimestampHandlerBeforeAbsentBucketIntervalHandler() { DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); - InputRow inputRow = Factory.createInputRow(invalidTimestamp); - CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); + InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(invalidTimestamp); + CloseableIterator inputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow); GranularitySpec absentBucketIntervalGranularitySpec = - Factory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); + IndexTaskInputRowIteratorBuilderTestingFactory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); exception.expect(ParseException.class); exception.expectMessage("Encountered row with timestamp that cannot be represented as a long"); @@ -160,19 +168,24 @@ public void invokesInvalidTimestampHandlerBeforeAbsentBucketIntervalHandler() @Test public void invokesAbsentBucketIntervalHandlerLast() { - DateTime timestamp = Factory.TIMESTAMP; - InputRow inputRow = Factory.createInputRow(timestamp); - CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); - GranularitySpec absentBucketIntervalGranularitySpec = Factory.createAbsentBucketIntervalGranularitySpec(timestamp); - - List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( - inputRowIterator, - absentBucketIntervalGranularitySpec, - NO_NEXT_INPUT_ROW - ); + DateTime timestamp = IndexTaskInputRowIteratorBuilderTestingFactory.TIMESTAMP; + InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(timestamp); + CloseableIterator inputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow); + GranularitySpec absentBucketIntervalGranularitySpec = + IndexTaskInputRowIteratorBuilderTestingFactory.createAbsentBucketIntervalGranularitySpec(timestamp); + + List handlerInvocationHistory = + HANDLER_TESTER.invokeHandlers( + inputRowIterator, + absentBucketIntervalGranularitySpec, + NO_NEXT_INPUT_ROW + ); Assert.assertEquals( - Collections.singletonList(Factory.HandlerTester.Handler.ABSENT_BUCKET_INTERVAL), + Collections.singletonList( + IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.ABSENT_BUCKET_INTERVAL + ), handlerInvocationHistory ); } @@ -180,22 +193,27 @@ public void invokesAbsentBucketIntervalHandlerLast() @Test public void invokesAppendedHandlersLast() { - DateTime timestamp = Factory.TIMESTAMP; - InputRow inputRow = Factory.createInputRow(timestamp); - CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); - GranularitySpec granularitySpec = Factory.createGranularitySpec(timestamp, Factory.PRESENT_BUCKET_INTERVAL_OPT); + DateTime timestamp = IndexTaskInputRowIteratorBuilderTestingFactory.TIMESTAMP; + InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(timestamp); + CloseableIterator inputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow); + GranularitySpec granularitySpec = IndexTaskInputRowIteratorBuilderTestingFactory.createGranularitySpec( + timestamp, + IndexTaskInputRowIteratorBuilderTestingFactory.PRESENT_BUCKET_INTERVAL_OPT + ); List appendedHandlers = Collections.singletonList(row -> true); - List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers( - inputRowIterator, - granularitySpec, - appendedHandlers, - NO_NEXT_INPUT_ROW - ); + List handlerInvocationHistory = + HANDLER_TESTER.invokeHandlers( + inputRowIterator, + granularitySpec, + appendedHandlers, + NO_NEXT_INPUT_ROW + ); Assert.assertEquals( - Collections.singletonList(Factory.HandlerTester.Handler.APPENDED), + Collections.singletonList(IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.APPENDED), handlerInvocationHistory ); } @@ -204,11 +222,15 @@ public void invokesAppendedHandlersLast() public void doesNotInvokeHandlersIfRowValid() { DateTime timestamp = DateTimes.utc(0); - InputRow inputRow = Factory.createInputRow(timestamp); - CloseableIterator inputRowIterator = Factory.createInputRowIterator(inputRow); - GranularitySpec granularitySpec = Factory.createGranularitySpec(timestamp, Factory.PRESENT_BUCKET_INTERVAL_OPT); + InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(timestamp); + CloseableIterator inputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow); + GranularitySpec granularitySpec = IndexTaskInputRowIteratorBuilderTestingFactory.createGranularitySpec( + timestamp, + IndexTaskInputRowIteratorBuilderTestingFactory.PRESENT_BUCKET_INTERVAL_OPT + ); - List handlerInvocationHistory = + List handlerInvocationHistory = HANDLER_TESTER.invokeHandlers(inputRowIterator, granularitySpec, inputRow); Assert.assertEquals(Collections.emptyList(), handlerInvocationHistory); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java similarity index 97% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java index b2bf4fd51dc8..754742fe3780 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/Factory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java @@ -36,7 +36,10 @@ import java.util.List; import java.util.function.Supplier; -class Factory +/** + * Helper for creating objects for testing {@link IndexTaskInputRowIteratorBuilder}s. + */ +class IndexTaskInputRowIteratorBuilderTestingFactory { static final DateTime TIMESTAMP = DateTimes.utc(0); static final String DIMENSION = "dimension";