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 new file mode 100644 index 000000000000..fa2c6d7235c0 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/HandlingInputRowIterator.java @@ -0,0 +1,92 @@ +/* + * 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.apache.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; + +/** + * Decorated {@link CloseableIterator} that can process rows with {@link InputRowHandler}s. + */ +public class HandlingInputRowIterator implements CloseableIterator +{ + @FunctionalInterface + public interface InputRowHandler + { + /** + * @return True if inputRow was successfully handled and no further processing is needed + */ + boolean handle(InputRow inputRow); + } + + private final CloseableIterator delegate; + private final List inputRowHandlers; + + /** + * @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 HandlingInputRowIterator( + CloseableIterator inputRowIterator, + List inputRowHandlers + ) + { + this.delegate = inputRowIterator; + this.inputRowHandlers = inputRowHandlers; + } + + @Override + public boolean hasNext() + { + return delegate.hasNext(); + } + + /** + * @return Next {@link InputRow} or null if row was successfully handled by an {@link InputRowHandler}. + */ + @Override + @Nullable + 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; + } + } + + return inputRow; + } + + @Override + public void close() throws IOException + { + delegate.close(); + } +} + 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..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 @@ -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,29 @@ static Integer resolveHistoricalNullIfNeeded(@Nullable Integer val) { return isEffectivelyNull(val) ? null : val; } + + /** + * @return True if this partitionSpec's type is compatible with forceGuaranteedRollup=true. + */ + @JsonIgnore + default boolean isForceGuaranteedRollupCompatibleType() + { + return !(this instanceof DynamicPartitionsSpec); + } + + /** + * @return True if this partitionSpec's property values are compatible with forceGuaranteedRollup=true. + */ + @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. + */ + @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/HandlingInputRowIteratorTest.java b/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java new file mode 100644 index 000000000000..8c907badabf1 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/HandlingInputRowIteratorTest.java @@ -0,0 +1,179 @@ +/* + * 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.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +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 HandlingInputRowIteratorTest +{ + public static class AbsentRowTest + { + 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; + + @Before + public void setup() + { + target = new HandlingInputRowIterator(EMPTY_ITERATOR, 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() + { + HandlingInputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); + Assert.assertTrue(target.hasNext()); + Assert.assertFalse(unsuccessfulHandler.invoked); + } + + @Test + public void yieldsNextIfUnhandled() + { + HandlingInputRowIterator target = createInputRowIterator(unsuccessfulHandler, unsuccessfulHandler); + Assert.assertEquals(INPUT_ROW1, target.next()); + Assert.assertTrue(unsuccessfulHandler.invoked); + } + + @Test + public void yieldsNullIfHandledByFirst() + { + HandlingInputRowIterator target = createInputRowIterator(successfulHandler, unsuccessfulHandler); + Assert.assertNull(target.next()); + Assert.assertTrue(successfulHandler.invoked); + Assert.assertFalse(unsuccessfulHandler.invoked); + } + + @Test + public void yieldsNullIfHandledBySecond() + { + HandlingInputRowIterator target = createInputRowIterator(unsuccessfulHandler, successfulHandler); + Assert.assertNull(target.next()); + Assert.assertTrue(unsuccessfulHandler.invoked); + Assert.assertTrue(successfulHandler.invoked); + } + + private static HandlingInputRowIterator createInputRowIterator( + HandlingInputRowIterator.InputRowHandler firstHandler, + HandlingInputRowIterator.InputRowHandler secondHandler + ) + { + 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)); + } + + private static class TestInputRowHandler implements HandlingInputRowIterator.InputRowHandler + { + boolean invoked = false; + + private final boolean successful; + + TestInputRowHandler(boolean successful) + { + this.successful = successful; + } + + @Override + public boolean handle(InputRow inputRow) + { + invoked = true; + return successful; + } + } + } +} 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..2a71a9a72c56 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -237,6 +237,11 @@ test-jar test + + org.hamcrest + hamcrest-all + test + org.hamcrest hamcrest-core 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..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 @@ -188,6 +188,12 @@ public abstract List findSegmentsToLock(TaskActionClient taskAction @Nullable public abstract Granularity getSegmentGranularity(); + @Override + public 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/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index 33e082b06761..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 @@ -19,62 +19,69 @@ 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 HashPartitionCachingLocalSegmentAllocator */ -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 +94,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().getJsonMapper(), i) - )) - .collect(Collectors.toList()) - ); - } - return intervalToSegmentIds; - } - private static String findVersion(Map intervalToVersion, Interval interval) { return intervalToVersion.entrySet().stream() @@ -128,27 +103,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/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 32bdf0248c34..33f7b9d3c527 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/HashPartitionCachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HashPartitionCachingLocalSegmentAllocator.java new file mode 100644 index 000000000000..9640ed461358 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HashPartitionCachingLocalSegmentAllocator.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 hash-partitioned segments locally at the beginning and reuses them. + * + * @see CachingLocalSegmentAllocator + */ +public class HashPartitionCachingLocalSegmentAllocator implements IndexTaskSegmentAllocator +{ + private final TaskToolbox toolbox; + private final String dataSource; + private final Map> allocateSpec; + private final IndexTaskSegmentAllocator delegate; + + public HashPartitionCachingLocalSegmentAllocator( + 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.getJsonMapper(), 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/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 3674c848a4ce..094a713cde16 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 @@ -65,6 +65,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.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -254,12 +255,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() { @@ -833,7 +828,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 HashPartitionCachingLocalSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); } else { return new LocalSegmentAllocator(toolbox, getId(), getDataSource(), dataSchema.getGranularitySpec()); } @@ -916,7 +911,8 @@ private TaskStatus generateAndPublishSegments( buildSegmentsSavedParseExceptions, tuningConfig.isLogParseExceptions(), tuningConfig.getMaxParseExceptions(), - pushTimeout + pushTimeout, + new DefaultIndexTaskInputRowIteratorBuilder() ); inputSourceProcessor.process( dataSchema, @@ -1000,26 +996,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) @@ -1223,7 +1205,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi @Nullable private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - public static IndexTuningConfig createDefault() + static IndexTuningConfig createDefault() { return new IndexTuningConfig(); } @@ -1256,8 +1238,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)) { 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 6d2dc0d75d1e..088503e0536a 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,6 +20,7 @@ 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.InputRowSchema; @@ -28,9 +29,8 @@ 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.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; @@ -63,13 +63,15 @@ public class InputSourceProcessor private final boolean logParseExceptions; private final int maxParseExceptions; private final long pushTimeout; + private final IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder; public InputSourceProcessor( RowIngestionMeters buildSegmentsMeters, @Nullable CircularBuffer buildSegmentsSavedParseExceptions, boolean logParseExceptions, int maxParseExceptions, - long pushTimeout + long pushTimeout, + IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder ) { this.buildSegmentsMeters = buildSegmentsMeters; @@ -77,6 +79,7 @@ public InputSourceProcessor( this.logParseExceptions = logParseExceptions; this.maxParseExceptions = maxParseExceptions; this.pushTimeout = pushTimeout; + this.inputRowIteratorBuilder = inputRowIteratorBuilder; } /** @@ -101,6 +104,7 @@ public SegmentsAndMetadata process( ? (DynamicPartitionsSpec) partitionsSpec : null; final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + final List metricsNames = Arrays.stream(dataSchema.getAggregators()) .map(AggregatorFactory::getName) .collect(Collectors.toList()); @@ -115,31 +119,27 @@ public SegmentsAndMetadata process( tmpDir ) ); - try (final CloseableIterator inputRowIterator = inputSourceReader.read()) { - while (inputRowIterator.hasNext()) { + 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 = inputRowIterator.next(); - + final 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; } + // IndexTaskInputRowIteratorBuilder.absentBucketIntervalConsumer() ensures the interval will be present here + Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + @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/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 4a511211a892..e2857d1fc553 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 @@ -28,8 +28,8 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.batch.parallel.LegacySinglePhaseSubTask; 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; @@ -58,8 +58,8 @@ @Type(name = ParallelIndexSupervisorTask.TYPE, value = ParallelIndexSupervisorTask.class), @Type(name = SinglePhaseSubTask.TYPE, value = SinglePhaseSubTask.class), @Type(name = SinglePhaseSubTask.OLD_TYPE_NAME, value = LegacySinglePhaseSubTask.class), // for backward compatibility - @Type(name = PartialSegmentGenerateTask.TYPE, value = PartialSegmentGenerateTask.class), - @Type(name = PartialSegmentMergeTask.TYPE, value = PartialSegmentMergeTask.class), + @Type(name = PartialHashSegmentGenerateTask.TYPE, value = PartialHashSegmentGenerateTask.class), + @Type(name = PartialHashSegmentMergeTask.TYPE, value = PartialHashSegmentMergeTask.class), @Type(name = "index_hadoop", value = HadoopIndexTask.class), @Type(name = "index_realtime", value = RealtimeIndexTask.class), @Type(name = "index_realtime_appenderator", value = AppenderatorDriverRealtimeIndexTask.class), 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..85574b7b254b --- /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}. + */ +class GeneratedHashPartitionsReport extends GeneratedPartitionsReport implements SubTaskReport +{ + public static final String TYPE = "generated_partitions"; + + @JsonCreator + 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 d1fd0000d13f..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; + 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; @@ -55,7 +48,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/PartialSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/InputSourceSplitParallelIndexTaskRunner.java similarity index 69% 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/InputSourceSplitParallelIndexTaskRunner.java index 3445eb192c26..c526ba7b8647 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/InputSourceSplitParallelIndexTaskRunner.java @@ -19,30 +19,29 @@ 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.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.SplittableInputSource; 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.InputSource} splits. */ -class PartialSegmentGenerateParallelIndexTaskRunner - extends ParallelIndexPhaseRunner +abstract class InputSourceSplitParallelIndexTaskRunner + extends ParallelIndexPhaseRunner { private final ParallelIndexIngestionSpec ingestionSchema; private final SplittableInputSource baseInputSource; - PartialSegmentGenerateParallelIndexTaskRunner( + InputSourceSplitParallelIndexTaskRunner( TaskToolbox toolbox, String taskId, String groupId, @@ -66,13 +65,7 @@ class PartialSegmentGenerateParallelIndexTaskRunner } @Override - public String getName() - { - return PartialSegmentGenerateTask.TYPE; - } - - @Override - Iterator> subTaskSpecIterator() throws IOException + Iterator> subTaskSpecIterator() throws IOException { return baseInputSource.createSplits( ingestionSchema.getIOConfig().getInputFormat(), @@ -81,7 +74,7 @@ Iterator> subTaskSpecIterator() throws I } @Override - int getTotalNumSubTasks() throws IOException + final int getTotalNumSubTasks() throws IOException { return baseInputSource.getNumSplits( ingestionSchema.getIOConfig().getInputFormat(), @@ -89,19 +82,7 @@ int getTotalNumSubTasks() throws IOException ); } - @VisibleForTesting - ParallelIndexIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - - @VisibleForTesting - SplittableInputSource getBaseInputSource() - { - return baseInputSource; - } - - SubTaskSpec newTaskSpec(InputSplit split) + final SubTaskSpec newTaskSpec(InputSplit split) { final FirehoseFactory firehoseFactory; final SplittableInputSource inputSource; @@ -122,30 +103,28 @@ 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() + ); } + + /** + * @return Ingestion spec split suitable for this parallel worker + */ + 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/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java index 7a92697b8eff..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 @@ -163,7 +163,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 b2e3c25c566d..28bfc7c421b1 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 @@ -34,6 +34,8 @@ import org.apache.druid.data.input.impl.InputRowParser; 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; @@ -102,6 +104,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; @@ -182,9 +185,7 @@ public ParallelIndexSupervisorTask( this.ingestionSchema = ingestionSchema; 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"); @@ -207,13 +208,15 @@ public ParallelIndexSupervisorTask( if (missingIntervalsInOverwriteMode) { addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } - } - @Override - public int getPriority() + private static void checkPartitionsSpecForForceGuaranteedRollup(PartitionsSpec partitionsSpec) { - return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); + if (!partitionsSpec.isForceGuaranteedRollupCompatible()) { + String incompatibiltyMsg = partitionsSpec.getForceGuaranteedRollupIncompatiblityReason(); + String msg = "forceGuaranteedRollup is incompatible with partitionsSpec: " + incompatibiltyMsg; + throw new ISE(msg); + } } @Override @@ -259,7 +262,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"); @@ -283,9 +286,9 @@ SinglePhaseParallelIndexTaskRunner createSinglePhaseTaskRunner(TaskToolbox toolb } @VisibleForTesting - public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerateRunner(TaskToolbox toolbox) + PartialHashSegmentGenerateParallelIndexTaskRunner createPartialHashSegmentGenerateRunner(TaskToolbox toolbox) { - return new PartialSegmentGenerateParallelIndexTaskRunner( + return new PartialHashSegmentGenerateParallelIndexTaskRunner( toolbox, getId(), getGroupId(), @@ -296,12 +299,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(), @@ -461,17 +464,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); @@ -482,18 +499,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 = createHashMergeIOConfigs( 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()) { @@ -504,40 +519,63 @@ 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 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()); @@ -554,23 +592,23 @@ private static List createMergeIOConfigs( 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; } @@ -786,7 +824,9 @@ static InputFormat getInputFormat(ParallelIndexIngestionSpec ingestionSchema) } /** - * {@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..db92233aab30 --- /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 InputSourceSplitParallelIndexTaskRunner +{ + // 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..d7f886207719 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -0,0 +1,173 @@ +/* + * 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.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; +import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; +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.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 HashedPartitionsSpec}. Partitioned segments are + * stored in local storage using {@link org.apache.druid.indexing.worker.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 HashPartitionCachingLocalSegmentAllocator( + toolbox, + getId(), + getDataSource(), + createShardSpecs() + ); + } + + @Override + 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(), + 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/PartialHashSegmentMergeIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java new file mode 100644 index 000000000000..2bc00ce3d9b6 --- /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) +class PartialHashSegmentMergeIOConfig extends PartialSegmentMergeIOConfig + implements IOConfig +{ + @JsonCreator + 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..abfef7608809 --- /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; + +class PartialHashSegmentMergeIngestionSpec extends PartialSegmentMergeIngestionSpec +{ + @JsonCreator + 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/PartialSegmentMergeParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java similarity index 68% 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..16b54ab71ffb 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(PartialHashSegmentMergeIOConfig ioConfig) { - final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec( - dataSchema, - ioConfig, - getTuningConfig() - ); - return new SubTaskSpec( + final PartialHashSegmentMergeIngestionSpec ingestionSpec = + new PartialHashSegmentMergeIngestionSpec( + 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..fa23eed2d1a5 --- /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 PartialHashSegmentMergeIngestionSpec 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 PartialHashSegmentMergeIngestionSpec 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 PartialHashSegmentMergeIngestionSpec 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.getJsonMapper() + ); + } +} 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 af1772fc3dc8..4dabf7ea02c5 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.InputSource; 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.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; import org.apache.druid.indexing.common.task.InputSourceProcessor; 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; @@ -68,143 +55,51 @@ import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -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 InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource( ingestionSchema.getDataSchema().getParser() @@ -223,23 +118,24 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); final List segments = generateSegments(toolbox, inputSource, tmpDir); - 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)); + taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments)); return TaskStatus.success(getId()); } + /** + * @return {@link IndexTaskSegmentAllocator} suitable for the desired segment partitioning strategy. + */ + abstract IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException; + + /** + * @return {@link GeneratedPartitionsReport} suitable for the desired segment partitioning strategy. + */ + abstract T createGeneratedPartitionsReport( + TaskToolbox toolbox, + List segments + ); + private List generateSegments( final TaskToolbox toolbox, final InputSource inputSource, @@ -247,7 +143,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), @@ -266,22 +161,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(), @@ -302,7 +185,8 @@ private List generateSegments( null, tuningConfig.isLogParseExceptions(), tuningConfig.getMaxParseExceptions(), - pushTimeout + pushTimeout, + inputRowIteratorBuilder ); final SegmentsAndMetadata pushed = inputSourceProcessor.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..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(PartialSegmentMergeTask.TYPE) -public class PartialSegmentMergeIOConfig implements IOConfig +abstract class PartialSegmentMergeIOConfig implements IOConfig { - private final List partitionLocations; + private final List partitionLocations; - @JsonCreator - public PartialSegmentMergeIOConfig(@JsonProperty("partitionLocations") List partitionLocations) + PartialSegmentMergeIOConfig(List partitionLocations) { Preconditions.checkState( partitionLocations != null && !partitionLocations.isEmpty(), @@ -43,7 +39,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..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 +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/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 4a10957c7e19..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,8 +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; import com.google.common.base.Preconditions; @@ -29,15 +27,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 +40,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 +50,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 +76,54 @@ 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( + 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 +132,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 +148,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 +187,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 +211,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception private Map>> fetchSegmentFiles( TaskToolbox toolbox, - Map>> intervalToPartitions + Map>> intervalToPartitions ) throws IOException { final File tempDir = toolbox.getIndexingTmpDir(); @@ -290,9 +220,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 +232,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 +254,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 +278,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 +300,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 +324,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.getJsonMapper() - ), + createShardSpec(toolbox, interval, partitionId), null, // will be filled in the segmentPusher 0 // will be filled in the segmentPusher ), @@ -408,7 +340,8 @@ private Set mergeAndPushSegments( } private static Pair> mergeSegmentsInSamePartition( - PartialSegmentMergeIngestionSpec ingestionSpec, + DataSchema dataSchema, + ParallelIndexTuningConfig tuningConfig, IndexIO indexIO, IndexMergerV9 merger, List indexes, @@ -439,11 +372,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 +387,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..c09049fde339 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 @@ -19,34 +19,33 @@ 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.java.util.common.StringUtils; 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 + PartitionLocation( + String host, + int port, + boolean useHttps, + String subTaskId, + Interval interval, + T secondaryPartition ) { this.host = host; @@ -54,7 +53,7 @@ public PartitionLocation( this.useHttps = useHttps; this.subTaskId = subTaskId; this.interval = interval; - this.partitionId = partitionId; + this.secondaryPartition = secondaryPartition; } @JsonProperty @@ -87,13 +86,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 +100,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 +138,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..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 @@ -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,71 @@ 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; } + /** + * @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 public boolean equals(Object o) { @@ -125,7 +123,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 +132,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..7cce1d32e833 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PerfectRollupWorkerTask.java @@ -0,0 +1,122 @@ +/* + * 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.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 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/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 762dc7dee605..8ea6fc916118 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 @@ -168,12 +168,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/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/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..3a8ad8ab566c --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilder.java @@ -0,0 +1,149 @@ +/* + * 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.HandlingInputRowIterator; +import org.apache.druid.data.input.InputRow; +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; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +/** + *

+ * 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.
+ *
+ *   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.
+ *
+ *   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.
+ * 
+ */ +public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRowIteratorBuilder +{ + private CloseableIterator delegate = null; + private GranularitySpec granularitySpec = null; + private HandlingInputRowIterator.InputRowHandler nullRowHandler = null; + private HandlingInputRowIterator.InputRowHandler absentBucketIntervalHandler = null; + private final List appendedInputRowHandlers = new ArrayList<>(); + + @Override + public DefaultIndexTaskInputRowIteratorBuilder delegate(CloseableIterator inputRowIterator) + { + this.delegate = inputRowIterator; + 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 HandlingInputRowIterator build() + { + 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() + .add(nullRowHandler) + .add(createInvalidTimestampHandler()) + .add(absentBucketIntervalHandler) + .addAll(appendedInputRowHandlers); + + return new HandlingInputRowIterator(delegate, handlersBuilder.build()); + } + + /** + * @param inputRowHandler Optionally, append this input row handler to the required ones. + */ + DefaultIndexTaskInputRowIteratorBuilder appendInputRowHandler(HandlingInputRowIterator.InputRowHandler inputRowHandler) + { + this.appendedInputRowHandlers.add(inputRowHandler); + return this; + } + + private HandlingInputRowIterator.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..80b4ea88615b --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilder.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package 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.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; + +import java.util.function.Consumer; + +public interface IndexTaskInputRowIteratorBuilder +{ + Runnable NOOP_RUNNABLE = () -> { + }; + + Consumer NOOP_CONSUMER = inputRow -> { + }; + + /** + * @param inputRowIterator Source of {@link InputRow}s. + */ + IndexTaskInputRowIteratorBuilder delegate(CloseableIterator inputRowIterator); + + /** + * @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); + + HandlingInputRowIterator build(); +} 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/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/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/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java new file mode 100644 index 000000000000..c45c5ce09212 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -0,0 +1,284 @@ +/* + * 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.impl.LocalInputSource; +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.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.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; +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; + private final boolean useInputFormatApi; + + AbstractMultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) + { + this.lockGranularity = lockGranularity; + this.useInputFormatApi = useInputFormatApi; + } + + @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, + File inputDir, + String filter, + DimensionBasedPartitionsSpec partitionsSpec + ) throws Exception + { + final ParallelIndexSupervisorTask task = newTask( + parseSpec, + interval, + inputDir, + filter, + 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, + File inputDir, + String filter, + DimensionBasedPartitionsSpec partitionsSpec + ) + { + GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ); + + 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, + 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 bf8da6ece522..20b6548275ca 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.File; import java.io.IOException; import java.util.Arrays; -import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -106,7 +105,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase private IntermediaryDataManager intermediaryDataManager; - protected void initializeIntermeidaryDataManager() throws IOException + protected void initializeIntermediaryDataManager() throws IOException { intermediaryDataManager = new IntermediaryDataManager( new WorkerConfig(), @@ -344,32 +343,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/GeneratedHashPartitionsReportTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java new file mode 100644 index 000000000000..1343b9425833 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.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.Before; +import org.junit.Test; + +import java.util.Collections; + +public class GeneratedHashPartitionsReportTest +{ + private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); + + private GeneratedHashPartitionsReport target; + + @Before + public void setup() + { + target = new GeneratedHashPartitionsReport( + "task-id", + Collections.singletonList( + new HashPartitionStat( + ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, + ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.PARTITION_ID, + ParallelIndexTestingFactory.NUM_ROWS, + ParallelIndexTestingFactory.SIZE_BYTES + ) + ) + ); + } + + @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/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java new file mode 100644 index 000000000000..5b60bdf7a610 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.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.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; +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.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 HashPartitionCachingLocalSegmentAllocatorTest +{ + 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 HashPartitionCachingLocalSegmentAllocator target; + + @Before + public void setup() throws IOException + { + TaskToolbox toolbox = createToolbox(); + target = new HashPartitionCachingLocalSegmentAllocator( + 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.getJsonMapper()).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(DateTimes.utc(timestamp)); + EasyMock.expect(inputRow.getTimestampFromEpoch()).andStubReturn(timestamp); + EasyMock.expect(inputRow.getDimension(DIMENSION)).andStubReturn(Collections.singletonList(DIMENSION)); + EasyMock.replay(inputRow); + return inputRow; + } +} 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..dffd9d52e8b2 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -0,0 +1,335 @@ +/* + * 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.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}, 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} + ); + } + + private File inputDir; + + public HashPartitionMultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) + { + super(lockGranularity, useInputFormatApi); + } + + @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"), + inputDir, + "test_*", + 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(PartialHashSegmentMergeIOConfig ioConfig) + { + final PartialHashSegmentMergeIngestionSpec ingestionSpec = + new PartialHashSegmentMergeIngestionSpec( + 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, + PartialHashSegmentMergeIngestionSpec 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..1eb6f867cd8e --- /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 = ParallelIndexTestingFactory.createObjectMapper(); + + private HashPartitionStat target; + + @Before + public void setup() + { + target = new HashPartitionStat( + ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, + ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.PARTITION_ID, + ParallelIndexTestingFactory.NUM_ROWS, + ParallelIndexTestingFactory.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 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/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 21ad0a4632c5..97c6954e9247 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 @@ -29,6 +29,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; @@ -46,6 +47,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; @@ -63,6 +65,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() { @@ -83,7 +86,7 @@ public void serde() throws IOException ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTaskBuilder() .ingestionSpec( new ParallelIndexIngestionSpecBuilder() - .inputIntervals(Collections.singletonList(Intervals.of("2018/2019"))) + .inputIntervals(INTERVALS) .build() ) .build(); @@ -112,11 +115,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; @@ -125,6 +128,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(); @@ -190,6 +231,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/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java new file mode 100644 index 000000000000..3d6e86aa01f2 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -0,0 +1,247 @@ +/* + * 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.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.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.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.http.client.HttpClient; +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.transform.TransformSpec; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +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; + +/** + * Helper for creating objects for testing parallel indexing. + */ +class ParallelIndexTestingFactory +{ + 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 = 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; + 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 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"; + + static final HashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new HashBasedNumberedShardSpec( + PARTITION_ID, + PARTITION_ID + 1, + Collections.singletonList("dim"), + ParallelIndexTestingFactory.NESTED_OBJECT_MAPPER + ); + + static ObjectMapper createObjectMapper() + { + return TEST_UTILS.getTestObjectMapper(); + } + + @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); + TimestampSpec timestampSpec = new TimestampSpec(SCHEMA_TIME, "auto", null); + DimensionsSpec dimensionsSpec = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION)), + null, + null + ); + + return new DataSchema( + DATASOURCE, + timestampSpec, + dimensionsSpec, + new AggregatorFactory[]{}, + granularitySpec, + TransformSpec.NONE, + null, + NESTED_OBJECT_MAPPER + ); + } + + static ParallelIndexIngestionSpec createIngestionSpec( + InputSource inputSource, + InputFormat inputFormat, + ParallelIndexTuningConfig tuningConfig, + DataSchema dataSchema + ) + { + ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(null, inputSource, inputFormat, 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 TASK_CLIENT_FACTORY; + } + + 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/PartialHashSegmentGenerateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java new file mode 100644 index 000000000000..81a1f06b1032 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.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.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 = 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; + + @Before + public void setup() + { + target = new PartialHashSegmentGenerateTask( + ParallelIndexTestingFactory.AUTOMATIC_ID, + ParallelIndexTestingFactory.GROUP_ID, + ParallelIndexTestingFactory.TASK_RESOURCE, + ParallelIndexTestingFactory.SUPERVISOR_TASK_ID, + ParallelIndexTestingFactory.NUM_ATTEMPTS, + INGESTION_SPEC, + ParallelIndexTestingFactory.CONTEXT, + ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT, + ParallelIndexTestingFactory.TASK_CLIENT_FACTORY, + ParallelIndexTestingFactory.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/PartialHashSegmentMergeIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java new file mode 100644 index 000000000000..413c34d9d668 --- /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 = ParallelIndexTestingFactory.createObjectMapper(); + private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( + ParallelIndexTestingFactory.HOST, + ParallelIndexTestingFactory.PORT, + ParallelIndexTestingFactory.USE_HTTPS, + ParallelIndexTestingFactory.SUBTASK_ID, + ParallelIndexTestingFactory.INTERVAL, + ParallelIndexTestingFactory.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..d734739a0146 --- /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 = ParallelIndexTestingFactory.createObjectMapper(); + private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( + 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)); + private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( + null, + 1, + Collections.emptyList() + ); + + private PartialHashSegmentMergeIngestionSpec target; + + @Before + public void setup() + { + target = new PartialHashSegmentMergeIngestionSpec( + ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), + IO_CONFIG, + new ParallelIndexTestingFactory.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 new file mode 100644 index 000000000000..8a733a2fd223 --- /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 = ParallelIndexTestingFactory.createObjectMapper(); + private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( + 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)); + private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( + null, + 1, + Collections.emptyList() + ); + private static final PartialHashSegmentMergeIngestionSpec INGESTION_SPEC = + new PartialHashSegmentMergeIngestionSpec( + ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), + IO_CONFIG, + new ParallelIndexTestingFactory.TuningConfigBuilder() + .partitionsSpec(PARTITIONS_SPEC) + .build() + ); + + private PartialHashSegmentMergeTask target; + + @Before + public void setup() + { + target = new PartialHashSegmentMergeTask( + ParallelIndexTestingFactory.AUTOMATIC_ID, + ParallelIndexTestingFactory.GROUP_ID, + ParallelIndexTestingFactory.TASK_RESOURCE, + ParallelIndexTestingFactory.SUPERVISOR_TASK_ID, + ParallelIndexTestingFactory.NUM_ATTEMPTS, + INGESTION_SPEC, + ParallelIndexTestingFactory.CONTEXT, + ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT, + ParallelIndexTestingFactory.TASK_CLIENT_FACTORY, + ParallelIndexTestingFactory.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..c5dd5edfa77b --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/DefaultIndexTaskInputRowIteratorBuilderTest.java @@ -0,0 +1,239 @@ +/* + * 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.HandlingInputRowIterator; +import org.apache.druid.data.input.InputRow; +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; +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 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 = + IndexTaskInputRowIteratorBuilder.NOOP_CONSUMER; + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void requiresDelegate() + { + exception.expect(NullPointerException.class); + exception.expectMessage("delegate 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() + .delegate(ITERATOR) + .nullRowRunnable(NULL_ROW_RUNNABLE) + .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) + .build(); + } + + @Test + public void requiresNullRowHandler() + { + exception.expect(NullPointerException.class); + exception.expectMessage("nullRowRunnable required"); + + new DefaultIndexTaskInputRowIteratorBuilder() + .delegate(ITERATOR) + .granularitySpec(GRANULARITY_SPEC) + .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) + .build(); + } + + @Test + public void requiresAbsentBucketIntervalHandler() + { + exception.expect(NullPointerException.class); + exception.expectMessage("absentBucketIntervalConsumer required"); + + new DefaultIndexTaskInputRowIteratorBuilder() + .delegate(ITERATOR) + .granularitySpec(GRANULARITY_SPEC) + .nullRowRunnable(NULL_ROW_RUNNABLE) + .build(); + } + + @Test + public void succeedsIfAllRequiredPresent() + { + new DefaultIndexTaskInputRowIteratorBuilder() + .delegate(ITERATOR) + .granularitySpec(GRANULARITY_SPEC) + .nullRowRunnable(NULL_ROW_RUNNABLE) + .absentBucketIntervalConsumer(ABSENT_BUCKET_INTERVAL_CONSUMER) + .build(); + } + } + + public static class HandlerTest + { + private static final IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester HANDLER_TESTER = + IndexTaskInputRowIteratorBuilderTestingFactory.createHandlerTester( + DefaultIndexTaskInputRowIteratorBuilder::new + ); + private static final InputRow NO_NEXT_INPUT_ROW = null; + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void invokesNullRowHandlerFirst() + { + DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); + CloseableIterator nullInputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(null); + GranularitySpec absentBucketIntervalGranularitySpec = + IndexTaskInputRowIteratorBuilderTestingFactory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); + + List handlerInvocationHistory = + HANDLER_TESTER.invokeHandlers( + nullInputRowIterator, + absentBucketIntervalGranularitySpec, + NO_NEXT_INPUT_ROW + ); + + Assert.assertEquals( + Collections.singletonList(IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.NULL_ROW), + handlerInvocationHistory + ); + } + + @Test + public void invokesInvalidTimestampHandlerBeforeAbsentBucketIntervalHandler() + { + DateTime invalidTimestamp = DateTimes.utc(Long.MAX_VALUE); + InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(invalidTimestamp); + CloseableIterator inputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow); + GranularitySpec absentBucketIntervalGranularitySpec = + IndexTaskInputRowIteratorBuilderTestingFactory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp); + + exception.expect(ParseException.class); + exception.expectMessage("Encountered row with timestamp that cannot be represented as a long"); + + HANDLER_TESTER.invokeHandlers(inputRowIterator, absentBucketIntervalGranularitySpec, NO_NEXT_INPUT_ROW); + } + + @Test + public void invokesAbsentBucketIntervalHandlerLast() + { + 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( + IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.ABSENT_BUCKET_INTERVAL + ), + handlerInvocationHistory + ); + } + + @Test + public void invokesAppendedHandlersLast() + { + 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 + ); + + Assert.assertEquals( + Collections.singletonList(IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.APPENDED), + handlerInvocationHistory + ); + } + + @Test + public void doesNotInvokeHandlersIfRowValid() + { + DateTime timestamp = DateTimes.utc(0); + InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(timestamp); + CloseableIterator inputRowIterator = + IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow); + GranularitySpec granularitySpec = IndexTaskInputRowIteratorBuilderTestingFactory.createGranularitySpec( + timestamp, + IndexTaskInputRowIteratorBuilderTestingFactory.PRESENT_BUCKET_INTERVAL_OPT + ); + + 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/IndexTaskInputRowIteratorBuilderTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java new file mode 100644 index 000000000000..754742fe3780 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/iterator/IndexTaskInputRowIteratorBuilderTestingFactory.java @@ -0,0 +1,162 @@ +/* + * 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.HandlingInputRowIterator; +import org.apache.druid.data.input.InputRow; +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.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +/** + * Helper for creating objects for testing {@link IndexTaskInputRowIteratorBuilder}s. + */ +class IndexTaskInputRowIteratorBuilderTestingFactory +{ + static final DateTime TIMESTAMP = DateTimes.utc(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 CloseableIterator createInputRowIterator(InputRow inputRow) + { + return new CloseableIterator() + { + @Override + public void close() + { + // nothing + } + + @Override + public boolean hasNext() + { + return true; + } + + @Override + public InputRow next() + { + return inputRow; + } + }; + } + + 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( + CloseableIterator inputRowIterator, + GranularitySpec granularitySpec, + InputRow expectedNextInputRow + ) + { + return invokeHandlers( + inputRowIterator, + granularitySpec, + Collections.emptyList(), + expectedNextInputRow + ); + } + + List invokeHandlers( + CloseableIterator inputRowIterator, + GranularitySpec granularitySpec, + List appendedHandlers, + InputRow expectedNextInputRow + ) + { + List handlerInvocationHistory = new ArrayList<>(); + IndexTaskInputRowIteratorBuilder iteratorBuilder = iteratorBuilderSupplier.get() + .delegate(inputRowIterator) + .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); + } + + HandlingInputRowIterator 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..8264767e601e 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -39,6 +39,8 @@ 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; @@ -389,4 +391,21 @@ 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) + { + try { + String serialized = objectMapper.writeValueAsString(object); + Object deserialized = objectMapper.readValue(serialized, object.getClass()); + 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 24d125910cc4..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,6 +42,7 @@ public class InlineFirehoseFactory implements FiniteFirehoseFactory