Refactor parallel indexing perfect rollup partitioning (#8852)

* Refactor parallel indexing perfect rollup partitioning

Refactoring to make it easier to later add range partitioning for
perfect rollup parallel indexing. This is accomplished by adding several
new base classes (e.g., PerfectRollupWorkerTask) and new classes for
encapsulating logic that needs to be changed for different partitioning
strategies (e.g., IndexTaskInputRowIteratorBuilder).

The code is functionally equivalent to before except for the following
small behavior changes:

1) PartialSegmentMergeTask: Previously, this task had a priority of
   DEFAULT_TASK_PRIORITY. It now has a priority of
   DEFAULT_BATCH_INDEX_TASK_PRIORITY (via the new PerfectRollupWorkerTask
   base class), since it is a batch index task.

2) ParallelIndexPhaseRunner: A decorator was added to
   subTaskSpecIterator to ensure the subtasks are generated with unique
   ids. Previously, only tests (i.e., MultiPhaseParallelIndexingTest)
   would have this decorator, but this behavior is desired for non-test
   code as well.

* Fix forbidden apis and pmd warnings

* Fix analyze dependencies warnings

* Fix IndexTask json and add IT diags

* Fix parallel index supervisor<->worker serde

* Fix TeamCity inspection errors/warnings

* Fix TeamCity inspection errors/warnings again

* Integrate changes with those from #8823

* Address review comments

* Address more review comments

* Fix forbidden apis

* Address more review comments
This commit is contained in:
Chi Cao Minh 2019-11-20 17:24:12 -08:00 committed by Gian Merlino
parent ac6d703814
commit ff6217365b
65 changed files with 3974 additions and 1270 deletions

View File

@ -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<InputRow>} that can process rows with {@link InputRowHandler}s.
*/
public class HandlingInputRowIterator implements CloseableIterator<InputRow>
{
@FunctionalInterface
public interface InputRowHandler
{
/**
* @return True if inputRow was successfully handled and no further processing is needed
*/
boolean handle(InputRow inputRow);
}
private final CloseableIterator<InputRow> delegate;
private final List<InputRowHandler> 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<InputRow> inputRowIterator,
List<InputRowHandler> 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();
}
}

View File

@ -81,6 +81,12 @@ public class DynamicPartitionsSpec implements PartitionsSpec
return false;
}
@Override
public String getForceGuaranteedRollupIncompatiblityReason()
{
return NAME + " partitions unsupported";
}
@Override
public boolean equals(Object o)
{

View File

@ -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 class HashedPartitionsSpec implements DimensionBasedPartitionsSpec
return partitionDimensions;
}
@Override
public String getForceGuaranteedRollupIncompatiblityReason()
{
return getNumShards() == null ? NUM_SHARDS + " must be specified" : FORCE_GUARANTEED_ROLLUP_COMPATIBLE;
}
@Override
public boolean equals(Object o)
{

View File

@ -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 @@ public interface PartitionsSpec
{
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();
}

View File

@ -39,10 +39,12 @@ import java.util.Objects;
*/
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 class SingleDimensionPartitionsSpec implements DimensionBasedPartitionsSp
return partitionDimension == null ? Collections.emptyList() : Collections.singletonList(partitionDimension);
}
@Override
public String getForceGuaranteedRollupIncompatiblityReason()
{
return NAME + " partitions unsupported";
}
@Override
public boolean needsDeterminePartitions(boolean useForHadoopTask)
{

View File

@ -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<InputRow> EMPTY_ITERATOR = CloseableIterators.withEmptyBaggage(
new Iterator<InputRow>()
{
@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<InputRow> 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<InputRow> iterator = CloseableIterators.withEmptyBaggage(
new Iterator<InputRow>()
{
private final Iterator<InputRow> 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;
}
}
}
}

View File

@ -49,7 +49,7 @@ public class LoggerCaptureRule extends ExternalResource
@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 @@ public class LoggerCaptureRule extends ExternalResource
{
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<LogEvent> 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<LogEvent> getLogEvents()

View File

@ -237,6 +237,11 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-all</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-core</artifactId>

View File

@ -188,6 +188,12 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
@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;

View File

@ -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<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec;
@Nullable
private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
private final ShardSpecs shardSpecs;
// sequenceName -> segmentId
private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
@FunctionalInterface
interface IntervalToSegmentIdsCreator
{
/**
* @param versionFinder Returns the version for the specified interval
* @return Information for segment preallocation
*/
Map<Interval, List<SegmentIdWithShardSpec>> create(Function<Interval, String> versionFinder);
}
public CachingLocalSegmentAllocator(
CachingLocalSegmentAllocator(
TaskToolbox toolbox,
String taskId,
String dataSource,
Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec
IntervalToSegmentIdsCreator intervalToSegmentIdsCreator
) throws IOException
{
this.toolbox = toolbox;
this.taskId = taskId;
this.dataSource = dataSource;
this.allocateSpec = allocateSpec;
this.sequenceNameToSegmentId = new HashMap<>();
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToIds = getIntervalToSegmentIds();
final Map<Interval, String> intervalToVersion = toolbox.getTaskActionClient()
.submit(new LockListAction())
.stream()
.collect(Collectors.toMap(
TaskLock::getInterval,
TaskLock::getVersion
));
Function<Interval, String> versionFinder = interval -> findVersion(intervalToVersion, interval);
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToIds = intervalToSegmentIdsCreator.create(versionFinder);
final Map<Interval, List<ShardSpec>> shardSpecMap = new HashMap<>();
for (Map.Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) {
for (Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) {
final Interval interval = entry.getKey();
final List<SegmentIdWithShardSpec> idsPerInterval = intervalToIds.get(interval);
@ -87,38 +94,6 @@ public class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
shardSpecs = new ShardSpecs(shardSpecMap);
}
private Map<Interval, List<SegmentIdWithShardSpec>> getIntervalToSegmentIds() throws IOException
{
final Map<Interval, String> intervalToVersion = getToolbox().getTaskActionClient()
.submit(new LockListAction())
.stream()
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = getAllocateSpec();
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToSegmentIds = new HashMap<>(allocateSpec.size());
for (Entry<Interval, Pair<ShardSpecFactory, Integer>> 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<Interval, String> intervalToVersion, Interval interval)
{
return intervalToVersion.entrySet().stream()
@ -128,27 +103,6 @@ public class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
.orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
}
TaskToolbox getToolbox()
{
return toolbox;
}
String getTaskId()
{
return taskId;
}
String getDataSource()
{
return dataSource;
}
Map<Interval, Pair<ShardSpecFactory, Integer>> getAllocateSpec()
{
return allocateSpec;
}
@Override
public SegmentIdWithShardSpec allocate(
InputRow row,

View File

@ -179,12 +179,6 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
this.ingestionState = IngestionState.NOT_STARTED;
}
@Override
public int getPriority()
{
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
}
@Override
public String getType()
{

View File

@ -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<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec;
private final IndexTaskSegmentAllocator delegate;
public HashPartitionCachingLocalSegmentAllocator(
TaskToolbox toolbox,
String taskId,
String dataSource,
Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec
) throws IOException
{
this.toolbox = toolbox;
this.dataSource = dataSource;
this.allocateSpec = allocateSpec;
this.delegate = new CachingLocalSegmentAllocator(
toolbox,
taskId,
this::getIntervalToSegmentIds
);
}
private Map<Interval, List<SegmentIdWithShardSpec>> getIntervalToSegmentIds(Function<Interval, String> versionFinder)
{
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToSegmentIds =
Maps.newHashMapWithExpectedSize(allocateSpec.size());
for (Entry<Interval, Pair<ShardSpecFactory, Integer>> 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);
}
}

View File

@ -65,6 +65,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
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 class IndexTask extends AbstractBatchIndexTask implements ChatHandler
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 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
// 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 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
buildSegmentsSavedParseExceptions,
tuningConfig.isLogParseExceptions(),
tuningConfig.getMaxParseExceptions(),
pushTimeout
pushTimeout,
new DefaultIndexTaskInputRowIteratorBuilder()
);
inputSourceProcessor.process(
dataSchema,
@ -1000,26 +996,12 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
this.map = map;
}
/**
* Return the underlying map.
*
* @return a map of intervals to shardSpecs
*/
Map<Interval, List<ShardSpec>> getMap()
{
return map;
}
Set<Interval> 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 class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
public static IndexTuningConfig createDefault()
static IndexTuningConfig createDefault()
{
return new IndexTuningConfig();
}
@ -1256,8 +1238,8 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
}
} 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)) {

View File

@ -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.data.input.InputSourceReader;
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<Throwable> buildSegmentsSavedParseExceptions,
boolean logParseExceptions,
int maxParseExceptions,
long pushTimeout
long pushTimeout,
IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder
)
{
this.buildSegmentsMeters = buildSegmentsMeters;
@ -77,6 +79,7 @@ public class InputSourceProcessor
this.logParseExceptions = logParseExceptions;
this.maxParseExceptions = maxParseExceptions;
this.pushTimeout = pushTimeout;
this.inputRowIteratorBuilder = inputRowIteratorBuilder;
}
/**
@ -101,6 +104,7 @@ public class InputSourceProcessor
? (DynamicPartitionsSpec) partitionsSpec
: null;
final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
final List<String> metricsNames = Arrays.stream(dataSchema.getAggregators())
.map(AggregatorFactory::getName)
.collect(Collectors.toList());
@ -115,31 +119,27 @@ public class InputSourceProcessor
tmpDir
)
);
try (final CloseableIterator<InputRow> inputRowIterator = inputSourceReader.read()) {
while (inputRowIterator.hasNext()) {
try (
final CloseableIterator<InputRow> 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<Interval> optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp());
if (!optInterval.isPresent()) {
buildSegmentsMeters.incrementThrownAway();
continue;
}
// IndexTaskInputRowIteratorBuilder.absentBucketIntervalConsumer() ensures the interval will be present here
Optional<Interval> 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);

View File

@ -28,8 +28,8 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.batch.parallel.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 @@ import java.util.Map;
@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),

View File

@ -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<HashPartitionStat> implements SubTaskReport
{
public static final String TYPE = "generated_partitions";
@JsonCreator
GeneratedHashPartitionsReport(
@JsonProperty("taskId") String taskId,
@JsonProperty("partitionStats") List<HashPartitionStat> partitionStats
)
{
super(taskId, partitionStats);
}
}

View File

@ -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 @@ import java.util.Objects;
/**
* 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<T extends PartitionStat> implements SubTaskReport
{
public static final String TYPE = "generated_partitions";
private final String taskId;
private final List<PartitionStat> partitionStats;
private final List<T> partitionStats;
@JsonCreator
public GeneratedPartitionsReport(
@JsonProperty("taskId") String taskId,
@JsonProperty("partitionStats") List<PartitionStat> partitionStats
)
GeneratedPartitionsReport(String taskId, List<T> partitionStats)
{
this.taskId = taskId;
this.partitionStats = partitionStats;
@ -55,7 +48,7 @@ public class GeneratedPartitionsReport implements SubTaskReport
}
@JsonProperty
public List<PartitionStat> getPartitionStats()
public List<T> getPartitionStats()
{
return partitionStats;
}

View File

@ -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<Integer>
{
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;
}
}

View File

@ -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<Integer>
{
// 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);
}
}

View File

@ -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<PartialSegmentGenerateTask, GeneratedPartitionsReport>
abstract class InputSourceSplitParallelIndexTaskRunner<T extends Task, R extends SubTaskReport>
extends ParallelIndexPhaseRunner<T, R>
{
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<SubTaskSpec<PartialSegmentGenerateTask>> subTaskSpecIterator() throws IOException
Iterator<SubTaskSpec<T>> subTaskSpecIterator() throws IOException
{
return baseInputSource.createSplits(
ingestionSchema.getIOConfig().getInputFormat(),
@ -81,7 +74,7 @@ class PartialSegmentGenerateParallelIndexTaskRunner
}
@Override
int getTotalNumSubTasks() throws IOException
final int getTotalNumSubTasks() throws IOException
{
return baseInputSource.getNumSplits(
ingestionSchema.getIOConfig().getInputFormat(),
@ -89,19 +82,7 @@ class PartialSegmentGenerateParallelIndexTaskRunner
);
}
@VisibleForTesting
ParallelIndexIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@VisibleForTesting
SplittableInputSource<?> getBaseInputSource()
{
return baseInputSource;
}
SubTaskSpec<PartialSegmentGenerateTask> newTaskSpec(InputSplit split)
final SubTaskSpec<T> newTaskSpec(InputSplit split)
{
final FirehoseFactory firehoseFactory;
final SplittableInputSource inputSource;
@ -122,30 +103,28 @@ class PartialSegmentGenerateParallelIndexTaskRunner
),
ingestionSchema.getTuningConfig()
);
return new SubTaskSpec<PartialSegmentGenerateTask>(
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<T> createSubTaskSpec(
String id,
String groupId,
String supervisorTaskId,
Map<String, Object> context,
InputSplit split,
ParallelIndexIngestionSpec subTaskIngestionSpec,
IndexingServiceClient indexingServiceClient
);
}

View File

@ -163,7 +163,7 @@ public abstract class ParallelIndexPhaseRunner<SubTaskType extends Task, SubTask
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

View File

@ -34,6 +34,8 @@ import org.apache.druid.data.input.InputSource;
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.SortedSet;
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 class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
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 class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
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 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
}
}
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 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
}
@VisibleForTesting
public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerateRunner(TaskToolbox toolbox)
PartialHashSegmentGenerateParallelIndexTaskRunner createPartialHashSegmentGenerateRunner(TaskToolbox toolbox)
{
return new PartialSegmentGenerateParallelIndexTaskRunner(
return new PartialHashSegmentGenerateParallelIndexTaskRunner(
toolbox,
getId(),
getGroupId(),
@ -296,12 +299,12 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
}
@VisibleForTesting
public PartialSegmentMergeParallelIndexTaskRunner createPartialSegmentMergeRunner(
PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner(
TaskToolbox toolbox,
List<PartialSegmentMergeIOConfig> ioConfigs
List<PartialHashSegmentMergeIOConfig> ioConfigs
)
{
return new PartialSegmentMergeParallelIndexTaskRunner(
return new PartialHashSegmentMergeParallelIndexTaskRunner(
toolbox,
getId(),
getGroupId(),
@ -461,17 +464,31 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
*
* - 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<PartialSegmentGenerateTask, GeneratedPartitionsReport> indexingRunner = createRunner(
ParallelIndexTaskRunner<PartialHashSegmentGenerateTask, GeneratedHashPartitionsReport> indexingRunner = createRunner(
toolbox,
this::createPartialSegmentGenerateRunner
this::createPartialHashSegmentGenerateRunner
);
TaskState state = runNextPhase(indexingRunner);
@ -482,18 +499,16 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
// 2. Partial segment merge phase
// partition (interval, partitionId) -> partition locations
//noinspection ConstantConditions
Map<Pair<Interval, Integer>, List<PartitionLocation>> partitionToLocations = groupPartitionLocationsPerPartition(
indexingRunner.getReports()
);
final List<PartialSegmentMergeIOConfig> ioConfigs = createMergeIOConfigs(
Map<Pair<Interval, Integer>, List<HashPartitionLocation>> partitionToLocations =
groupHashPartitionLocationsPerPartition(indexingRunner.getReports());
final List<PartialHashSegmentMergeIOConfig> ioConfigs = createHashMergeIOConfigs(
ingestionSchema.getTuningConfig().getTotalNumMergeTasks(),
partitionToLocations
);
final ParallelIndexTaskRunner<PartialSegmentMergeTask, PushedSegmentsReport> mergeRunner = createRunner(
final ParallelIndexTaskRunner<PartialHashSegmentMergeTask, PushedSegmentsReport> mergeRunner = createRunner(
toolbox,
tb -> createPartialSegmentMergeRunner(tb, ioConfigs)
tb -> createPartialHashSegmentMergeRunner(tb, ioConfigs)
);
state = runNextPhase(mergeRunner);
if (state.isSuccess()) {
@ -504,40 +519,63 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
return TaskStatus.fromCode(getId(), state);
}
private static Map<Pair<Interval, Integer>, List<PartitionLocation>> groupPartitionLocationsPerPartition(
// subTaskId -> report
Map<String, GeneratedPartitionsReport> reports
private static Map<Pair<Interval, Integer>, List<HashPartitionLocation>> groupHashPartitionLocationsPerPartition(
Map<String, GeneratedHashPartitionsReport> subTaskIdToReport
)
{
// partition (interval, partitionId) -> partition locations
final Map<Pair<Interval, Integer>, List<PartitionLocation>> partitionToLocations = new HashMap<>();
for (Entry<String, GeneratedPartitionsReport> entry : reports.entrySet()) {
final String subTaskId = entry.getKey();
final GeneratedPartitionsReport report = entry.getValue();
for (PartitionStat partitionStat : report.getPartitionStats()) {
final List<PartitionLocation> locationsOfSamePartition = partitionToLocations.computeIfAbsent(
Pair.of(partitionStat.getInterval(), partitionStat.getPartitionId()),
k -> new ArrayList<>()
);
locationsOfSamePartition.add(
new PartitionLocation(
BiFunction<String, HashPartitionStat, HashPartitionLocation> 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 <S extends PartitionStat, L extends PartitionLocation>
Map<Pair<Interval, Integer>, List<L>> groupPartitionLocationsPerPartition(
Map<String, ? extends GeneratedPartitionsReport<S>> subTaskIdToReport,
BiFunction<String, S, L> createPartitionLocationFunction
)
{
// partition (interval, partitionId) -> partition locations
final Map<Pair<Interval, Integer>, List<L>> partitionToLocations = new HashMap<>();
for (Entry<String, ? extends GeneratedPartitionsReport<S>> entry : subTaskIdToReport.entrySet()) {
final String subTaskId = entry.getKey();
final GeneratedPartitionsReport<S> report = entry.getValue();
for (S partitionStat : report.getPartitionStats()) {
final List<L> locationsOfSamePartition = partitionToLocations.computeIfAbsent(
Pair.of(partitionStat.getInterval(), partitionStat.getPartitionId()),
k -> new ArrayList<>()
);
locationsOfSamePartition.add(createPartitionLocationFunction.apply(subTaskId, partitionStat));
}
}
return partitionToLocations;
}
private static List<PartialSegmentMergeIOConfig> createMergeIOConfigs(
private static List<PartialHashSegmentMergeIOConfig> createHashMergeIOConfigs(
int totalNumMergeTasks,
Map<Pair<Interval, Integer>, List<PartitionLocation>> partitionToLocations
Map<Pair<Interval, Integer>, List<HashPartitionLocation>> partitionToLocations
)
{
return createMergeIOConfigs(
totalNumMergeTasks,
partitionToLocations,
PartialHashSegmentMergeIOConfig::new
);
}
private static <M extends PartialSegmentMergeIOConfig, L extends PartitionLocation> List<M> createMergeIOConfigs(
int totalNumMergeTasks,
Map<Pair<Interval, Integer>, List<L>> partitionToLocations,
Function<List<L>, M> createPartialSegmentMergeIOConfig
)
{
final int numMergeTasks = Math.min(totalNumMergeTasks, partitionToLocations.size());
@ -554,23 +592,23 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
Collections.shuffle(partitions, ThreadLocalRandom.current());
final int numPartitionsPerTask = (int) Math.round(partitions.size() / (double) numMergeTasks);
final List<PartialSegmentMergeIOConfig> assignedPartitionLocations = new ArrayList<>(numMergeTasks);
final List<M> assignedPartitionLocations = new ArrayList<>(numMergeTasks);
for (int i = 0; i < numMergeTasks - 1; i++) {
final List<PartitionLocation> assignedToSameTask = partitions
final List<L> 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<PartitionLocation> assignedToSameTask = partitions
final List<L> 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 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
}
/**
* {@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")

View File

@ -38,7 +38,7 @@ import java.util.Set;
* 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<SubTaskType extends Task, SubTaskReportType extends SubTaskReport>

View File

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

View File

@ -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<PartialHashSegmentGenerateTask, GeneratedHashPartitionsReport>
{
// For tests
private final IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory;
private final AppenderatorsManager appenderatorsManager;
PartialHashSegmentGenerateParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
)
{
this(toolbox, taskId, groupId, ingestionSchema, context, indexingServiceClient, null, null);
}
@VisibleForTesting
PartialHashSegmentGenerateParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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<PartialHashSegmentGenerateTask> createSubTaskSpec(
String id,
String groupId,
String supervisorTaskId,
Map<String, Object> context,
InputSplit split,
ParallelIndexIngestionSpec subTaskIngestionSpec,
IndexingServiceClient indexingServiceClient
)
{
return new SubTaskSpec<PartialHashSegmentGenerateTask>(
id,
groupId,
supervisorTaskId,
context,
split
)
{
@Override
public PartialHashSegmentGenerateTask newSubTask(int numAttempts)
{
return new PartialHashSegmentGenerateTask(
null,
groupId,
null,
supervisorTaskId,
numAttempts,
subTaskIngestionSpec,
context,
indexingServiceClient,
taskClientFactory,
appenderatorsManager
);
}
};
}
}

View File

@ -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<GeneratedHashPartitionsReport>
{
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<String, Object> context,
@JacksonInject IndexingServiceClient indexingServiceClient,
@JacksonInject IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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<DataSegment> segments)
{
List<HashPartitionStat> 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<Interval, Pair<ShardSpecFactory, Integer>> createShardSpecs()
{
GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
return createShardSpecWithoutInputScan(
granularitySpec,
ingestionSchema.getIOConfig(),
tuningConfig,
partitionsSpec
);
}
}

View File

@ -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<HashPartitionLocation>
implements IOConfig
{
@JsonCreator
PartialHashSegmentMergeIOConfig(
@JsonProperty("partitionLocations") List<HashPartitionLocation> partitionLocations
)
{
super(partitionLocations);
}
}

View File

@ -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<PartialHashSegmentMergeIOConfig>
{
@JsonCreator
PartialHashSegmentMergeIngestionSpec(
@JsonProperty("dataSchema") DataSchema dataSchema,
@JsonProperty("ioConfig") PartialHashSegmentMergeIOConfig ioConfig,
@JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig
)
{
super(dataSchema, ioConfig, tuningConfig);
}
}

View File

@ -30,22 +30,22 @@ import java.util.List;
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<PartialSegmentMergeTask, PushedSegmentsReport>
class PartialHashSegmentMergeParallelIndexTaskRunner
extends ParallelIndexPhaseRunner<PartialHashSegmentMergeTask, PushedSegmentsReport>
{
private final DataSchema dataSchema;
private final List<PartialSegmentMergeIOConfig> mergeIOConfigs;
private final List<PartialHashSegmentMergeIOConfig> mergeIOConfigs;
PartialSegmentMergeParallelIndexTaskRunner(
PartialHashSegmentMergeParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
DataSchema dataSchema,
List<PartialSegmentMergeIOConfig> mergeIOConfigs,
List<PartialHashSegmentMergeIOConfig> mergeIOConfigs,
ParallelIndexTuningConfig tuningConfig,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
@ -60,11 +60,11 @@ class PartialSegmentMergeParallelIndexTaskRunner
@Override
public String getName()
{
return PartialSegmentMergeTask.TYPE;
return PartialHashSegmentMergeTask.TYPE;
}
@Override
Iterator<SubTaskSpec<PartialSegmentMergeTask>> subTaskSpecIterator()
Iterator<SubTaskSpec<PartialHashSegmentMergeTask>> subTaskSpecIterator()
{
return mergeIOConfigs.stream().map(this::newTaskSpec).iterator();
}
@ -76,14 +76,15 @@ class PartialSegmentMergeParallelIndexTaskRunner
}
@VisibleForTesting
SubTaskSpec<PartialSegmentMergeTask> newTaskSpec(PartialSegmentMergeIOConfig ioConfig)
SubTaskSpec<PartialHashSegmentMergeTask> newTaskSpec(PartialHashSegmentMergeIOConfig ioConfig)
{
final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec(
dataSchema,
ioConfig,
getTuningConfig()
);
return new SubTaskSpec<PartialSegmentMergeTask>(
final PartialHashSegmentMergeIngestionSpec ingestionSpec =
new PartialHashSegmentMergeIngestionSpec(
dataSchema,
ioConfig,
getTuningConfig()
);
return new SubTaskSpec<PartialHashSegmentMergeTask>(
getTaskId() + "_" + getAndIncrementNextSpecId(),
getGroupId(),
getTaskId(),
@ -92,9 +93,9 @@ class PartialSegmentMergeParallelIndexTaskRunner
)
{
@Override
public PartialSegmentMergeTask newSubTask(int numAttempts)
public PartialHashSegmentMergeTask newSubTask(int numAttempts)
{
return new PartialSegmentMergeTask(
return new PartialHashSegmentMergeTask(
null,
getGroupId(),
null,

View File

@ -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<HashBasedNumberedShardSpec, HashPartitionLocation>
{
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<String, Object> context,
@JacksonInject IndexingServiceClient indexingServiceClient,
@JacksonInject IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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()
);
}
}

View File

@ -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.AppenderatorsManager;
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.List;
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<T extends GeneratedPartitionsReport> 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<ParallelIndexSupervisorTaskClient> taskClientFactory;
private final AppenderatorsManager appenderatorsManager;
private final IndexTaskInputRowIteratorBuilder inputRowIteratorBuilder;
@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<String, Object> context,
@JacksonInject IndexingServiceClient indexingServiceClient,
@JacksonInject IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory,
@JacksonInject AppenderatorsManager appenderatorsManager
PartialSegmentGenerateTask(
String id,
String groupId,
TaskResource taskResource,
String supervisorTaskId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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;
this.inputRowIteratorBuilder = inputRowIteratorBuilder;
}
@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;
}
@Override
public List<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> 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 class PartialSegmentGenerateTask extends AbstractBatchIndexTask
);
final List<DataSegment> segments = generateSegments(toolbox, inputSource, tmpDir);
final List<PartitionStat> 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<DataSegment> segments
);
private List<DataSegment> generateSegments(
final TaskToolbox toolbox,
final InputSource inputSource,
@ -247,7 +143,6 @@ public class PartialSegmentGenerateTask extends AbstractBatchIndexTask
) 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 @@ public class PartialSegmentGenerateTask extends AbstractBatchIndexTask
}
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
final long pushTimeout = tuningConfig.getPushTimeout();
final Map<Interval, Pair<ShardSpecFactory, Integer>> 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 @@ public class PartialSegmentGenerateTask extends AbstractBatchIndexTask
null,
tuningConfig.isLogParseExceptions(),
tuningConfig.getMaxParseExceptions(),
pushTimeout
pushTimeout,
inputRowIteratorBuilder
);
final SegmentsAndMetadata pushed = inputSourceProcessor.process(
dataSchema,

View File

@ -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<T extends PartitionLocation> implements IOConfig
{
private final List<PartitionLocation> partitionLocations;
private final List<T> partitionLocations;
@JsonCreator
public PartialSegmentMergeIOConfig(@JsonProperty("partitionLocations") List<PartitionLocation> partitionLocations)
PartialSegmentMergeIOConfig(List<T> partitionLocations)
{
Preconditions.checkState(
partitionLocations != null && !partitionLocations.isEmpty(),
@ -43,7 +39,7 @@ public class PartialSegmentMergeIOConfig implements IOConfig
}
@JsonProperty
public List<PartitionLocation> getPartitionLocations()
public List<T> getPartitionLocations()
{
return partitionLocations;
}

View File

@ -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<PartialSegmentMergeIOConfig, ParallelIndexTuningConfig>
abstract class PartialSegmentMergeIngestionSpec<T extends PartialSegmentMergeIOConfig>
extends IngestionSpec<T, ParallelIndexTuningConfig>
{
@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);

View File

@ -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.Int2ObjectMap;
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.ISE;
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.IndexIO;
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.concurrent.TimeUnit;
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<S extends ShardSpec, P extends PartitionLocation> 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<P> ioConfig;
private final int numAttempts;
private final PartialSegmentMergeIngestionSpec ingestionSchema;
private final String supervisorTaskId;
private final IndexingServiceClient indexingServiceClient;
private final IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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<String, Object> context,
@JacksonInject IndexingServiceClient indexingServiceClient,
@JacksonInject IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory,
@JacksonInject @EscalatedClient HttpClient shuffleClient
@Nullable String id,
final String groupId,
final TaskResource taskResource,
final String supervisorTaskId,
DataSchema dataSchema,
PartialSegmentMergeIOConfig<P> ioConfig,
ParallelIndexTuningConfig tuningConfig,
final int numAttempts, // zero-based counting
final Map<String, Object> context,
IndexingServiceClient indexingServiceClient,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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 class PartialSegmentMergeTask extends AbstractBatchIndexTask
return numAttempts;
}
@JsonProperty("spec")
public PartialSegmentMergeIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@Override
public boolean requireLockExistingSegments()
{
return true;
}
@Override
public List<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> 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 class PartialSegmentMergeTask extends AbstractBatchIndexTask
public TaskStatus runTask(TaskToolbox toolbox) throws Exception
{
// Group partitionLocations by interval and partitionId
final Map<Interval, Int2ObjectMap<List<PartitionLocation>>> intervalToPartitions = new HashMap<>();
for (PartitionLocation location : ingestionSchema.getIOConfig().getPartitionLocations()) {
final Map<Interval, Int2ObjectMap<List<P>>> 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 class PartialSegmentMergeTask extends AbstractBatchIndexTask
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<DataSegment> pushedSegments = mergeAndPushSegments(
toolbox,
partitionsSpec,
getDataSchema(),
getTuningConfig(),
persistDir,
intervalToVersion,
intervalToUnzippedFiles
@ -281,7 +211,7 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
private Map<Interval, Int2ObjectMap<List<File>>> fetchSegmentFiles(
TaskToolbox toolbox,
Map<Interval, Int2ObjectMap<List<PartitionLocation>>> intervalToPartitions
Map<Interval, Int2ObjectMap<List<P>>> intervalToPartitions
) throws IOException
{
final File tempDir = toolbox.getIndexingTmpDir();
@ -290,9 +220,9 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
final Map<Interval, Int2ObjectMap<List<File>>> intervalToUnzippedFiles = new HashMap<>();
// Fetch partition files
for (Entry<Interval, Int2ObjectMap<List<PartitionLocation>>> entryPerInterval : intervalToPartitions.entrySet()) {
for (Entry<Interval, Int2ObjectMap<List<P>>> entryPerInterval : intervalToPartitions.entrySet()) {
final Interval interval = entryPerInterval.getKey();
for (Int2ObjectMap.Entry<List<PartitionLocation>> entryPerPartitionId :
for (Int2ObjectMap.Entry<List<P>> entryPerPartitionId :
entryPerInterval.getValue().int2ObjectEntrySet()) {
final int partitionId = entryPerPartitionId.getIntKey();
final File partitionDir = FileUtils.getFile(
@ -302,7 +232,7 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
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 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
}
@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 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
return zippedFile;
}
/**
* Create a {@link ShardSpec} suitable for the desired secondary partitioning strategy.
*/
abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int partitionNum);
private Set<DataSegment> mergeAndPushSegments(
TaskToolbox toolbox,
HashedPartitionsSpec partitionsSpec,
DataSchema dataSchema,
ParallelIndexTuningConfig tuningConfig,
File persistDir,
Map<Interval, String> intervalToVersion,
Map<Interval, Int2ObjectMap<List<File>>> intervalToUnzippedFiles
@ -364,15 +300,16 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
final int partitionId = entryPerPartitionId.getIntKey();
final List<File> segmentFilesToMerge = entryPerPartitionId.getValue();
final Pair<File, List<String>> mergedFileAndDimensionNames = mergeSegmentsInSamePartition(
ingestionSchema,
dataSchema,
tuningConfig,
toolbox.getIndexIO(),
toolbox.getIndexMergerV9(),
segmentFilesToMerge,
ingestionSchema.getTuningConfig().getMaxNumSegmentsToMerge(),
tuningConfig.getMaxNumSegmentsToMerge(),
persistDir,
0
);
final List<String> metricNames = Arrays.stream(ingestionSchema.getDataSchema().getAggregators())
final List<String> metricNames = Arrays.stream(dataSchema.getAggregators())
.map(AggregatorFactory::getName)
.collect(Collectors.toList());
@ -387,12 +324,7 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
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 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
}
private static Pair<File, List<String>> mergeSegmentsInSamePartition(
PartialSegmentMergeIngestionSpec ingestionSpec,
DataSchema dataSchema,
ParallelIndexTuningConfig tuningConfig,
IndexIO indexIO,
IndexMergerV9 merger,
List<File> indexes,
@ -439,11 +372,11 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
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 @@ public class PartialSegmentMergeTask extends AbstractBatchIndexTask
return Pair.of(mergedFiles.get(0), Preconditions.checkNotNull(dimensionNames, "dimensionNames"));
} else {
return mergeSegmentsInSamePartition(
ingestionSpec,
dataSchema,
tuningConfig,
indexIO,
merger,
mergedFiles,

View File

@ -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<T>
{
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 class PartitionLocation
this.useHttps = useHttps;
this.subTaskId = subTaskId;
this.interval = interval;
this.partitionId = partitionId;
this.secondaryPartition = secondaryPartition;
}
@JsonProperty
@ -87,13 +86,9 @@ public class PartitionLocation
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 @@ public class PartitionLocation
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 class PartitionLocation
", useHttps=" + useHttps +
", subTaskId='" + subTaskId + '\'' +
", interval=" + interval +
", partitionId=" + partitionId +
", secondaryPartition=" + secondaryPartition +
'}';
}
}

View File

@ -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 javax.annotation.Nullable;
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<T>
{
// 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 class PartitionStat
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 class PartitionStat
@Override
public int hashCode()
{
return Objects.hash(taskExecutorHost, taskExecutorPort, useHttps, interval, partitionId, numRows, sizeBytes);
return Objects.hash(taskExecutorHost, taskExecutorPort, useHttps, interval, numRows, sizeBytes);
}
}

View File

@ -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<String, Object> 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<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> 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;
}
}

View File

@ -168,12 +168,6 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
}
}
@Override
public int getPriority()
{
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
}
@Override
public String getType()
{

View File

@ -30,7 +30,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo.Id;
@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
{

View File

@ -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;
/**
* <pre>
* 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.
* </pre>
*/
public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRowIteratorBuilder
{
private CloseableIterator<InputRow> delegate = null;
private GranularitySpec granularitySpec = null;
private HandlingInputRowIterator.InputRowHandler nullRowHandler = null;
private HandlingInputRowIterator.InputRowHandler absentBucketIntervalHandler = null;
private final List<HandlingInputRowIterator.InputRowHandler> appendedInputRowHandlers = new ArrayList<>();
@Override
public DefaultIndexTaskInputRowIteratorBuilder delegate(CloseableIterator<InputRow> 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<InputRow> absentBucketIntervalConsumer
)
{
this.absentBucketIntervalHandler = inputRow -> {
Optional<Interval> 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<HandlingInputRowIterator.InputRowHandler> handlersBuilder = ImmutableList.<HandlingInputRowIterator.InputRowHandler>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;
};
}
}

View File

@ -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<InputRow> NOOP_CONSUMER = inputRow -> {
};
/**
* @param inputRowIterator Source of {@link InputRow}s.
*/
IndexTaskInputRowIteratorBuilder delegate(CloseableIterator<InputRow> 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<InputRow> absentBucketIntervalConsumer);
HandlingInputRowIterator build();
}

View File

@ -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.segment.writeout.OffHeapMemorySegmentWriteOutMediumFacto
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<ParallelIndexSupervisorTaskClient> TASK_CLIENT_FACTORY = new NoopIndexTaskClientFactory<>();
public static final AppenderatorsManager APPENDERATORS_MANAGER = new TestAppenderatorsManager();
public static final HttpClient SHUFFLE_CLIENT = new HttpClient()
{
@Override
public <Intermediate, Final> ListenableFuture<Final> go(
Request request,
HttpResponseHandler<Intermediate, Final> handler
)
{
throw new UnsupportedOperationException();
}
@Override
public <Intermediate, Final> ListenableFuture<Final> go(
Request request,
HttpResponseHandler<Intermediate, Final> handler,
Duration readTimeout
)
{
throw new UnsupportedOperationException();
}
};
private static final Logger log = new Logger(TestUtils.class);
private final ObjectMapper jsonMapper;
@ -70,14 +105,7 @@ public class 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 class TestUtils
.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 class TestUtils
}
}
);
List<? extends Module> firehoseModules = new FirehoseModule().getJacksonModules();
firehoseModules.forEach(jsonMapper::registerModule);
}
public ObjectMapper getTestObjectMapper()

View File

@ -189,7 +189,7 @@ public class IndexTaskSerdeTest
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,

View File

@ -23,10 +23,10 @@ import org.apache.druid.indexing.common.IndexTaskClient;
import org.apache.druid.indexing.common.TaskInfoProvider;
import org.joda.time.Duration;
public class NoopIndexTaskClientFactory implements IndexTaskClientFactory
public class NoopIndexTaskClientFactory<T extends IndexTaskClient> implements IndexTaskClientFactory<T>
{
@Override
public IndexTaskClient build(
public T build(
TaskInfoProvider taskInfoProvider,
String callerId,
int numThreads,

View File

@ -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<DataSegment> 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<String, Object> context,
IndexingServiceClient indexingServiceClient
);
List<ScanResultValue> querySegment(DataSegment dataSegment, List<String> columns, File tempSegmentDir)
{
Segment segment = loadSegment(dataSegment, tempSegmentDir);
final QueryRunner<ScanResultValue> 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);
}
}
}

View File

@ -73,7 +73,6 @@ import javax.annotation.Nullable;
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 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
indexingServiceClient
);
}
@Override
Iterator<SubTaskSpec<SinglePhaseSubTask>> subTaskSpecIterator() throws IOException
{
final Iterator<SubTaskSpec<SinglePhaseSubTask>> iterator = super.subTaskSpecIterator();
return new Iterator<SubTaskSpec<SinglePhaseSubTask>>()
{
@Override
public boolean hasNext()
{
return iterator.hasNext();
}
@Override
public SubTaskSpec<SinglePhaseSubTask> next()
{
try {
Thread.sleep(10);
return iterator.next();
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
};
}
}
static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory<ParallelIndexSupervisorTaskClient>

View File

@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.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);
}
}

View File

@ -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<String> 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<Interval, Pair<ShardSpecFactory, Integer>> 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<TaskLock> 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;
}
}

View File

@ -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<Object[]> 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<DataSegment> publishedSegments = runTestTask(
PARSE_SPEC,
Intervals.of("2017/2018"),
inputDir,
"test_*",
new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2"))
);
assertHashedPartition(publishedSegments);
}
private void assertHashedPartition(Set<DataSegment> publishedSegments) throws IOException
{
final Map<Interval, List<DataSegment>> intervalToSegments = new HashMap<>();
publishedSegments.forEach(
segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment)
);
final File tempSegmentDir = temporaryFolder.newFolder();
for (List<DataSegment> segmentsInInterval : intervalToSegments.values()) {
Assert.assertEquals(2, segmentsInInterval.size());
for (DataSegment segment : segmentsInInterval) {
List<ScanResultValue> results = querySegment(segment, ImmutableList.of("dim1", "dim2"), tempSegmentDir);
final int hash = HashBasedNumberedShardSpec.hash(getObjectMapper(), (List<Object>) results.get(0).getEvents());
for (ScanResultValue value : results) {
Assert.assertEquals(
hash,
HashBasedNumberedShardSpec.hash(getObjectMapper(), (List<Object>) value.getEvents())
);
}
}
}
}
@Override
ParallelIndexSupervisorTask createParallelIndexSupervisorTask(
String id,
TaskResource taskResource,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> 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<String, Object> 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<PartialHashSegmentMergeIOConfig> 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<PartialHashSegmentMergeIOConfig> mergeIOConfigs,
IndexingServiceClient indexingServiceClient
)
{
super(
toolbox,
supervisorTask.getId(),
supervisorTask.getGroupId(),
supervisorTask.getIngestionSchema().getDataSchema(),
mergeIOConfigs,
supervisorTask.getIngestionSchema().getTuningConfig(),
supervisorTask.getContext(),
indexingServiceClient
);
this.supervisorTask = supervisorTask;
}
@Override
SubTaskSpec<PartialHashSegmentMergeTask> newTaskSpec(PartialHashSegmentMergeIOConfig ioConfig)
{
final PartialHashSegmentMergeIngestionSpec ingestionSpec =
new PartialHashSegmentMergeIngestionSpec(
supervisorTask.getIngestionSchema().getDataSchema(),
ioConfig,
getTuningConfig()
);
return new SubTaskSpec<PartialHashSegmentMergeTask>(
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<String, Object> context,
IndexingServiceClient indexingServiceClient,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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;
}
}
}

View File

@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.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());
}
}

View File

@ -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<Object[]> 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<DataSegment> publishedSegments = runTestTask(
Intervals.of("2017/2018"),
new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2"))
);
assertHashedPartition(publishedSegments);
}
private Set<DataSegment> 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<DataSegment> publishedSegments) throws IOException, SegmentLoadingException
{
final Map<Interval, List<DataSegment>> intervalToSegments = new HashMap<>();
publishedSegments.forEach(
segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment)
);
final File tempSegmentDir = temporaryFolder.newFolder();
for (List<DataSegment> 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<ScanResultValue> runner = factory.createRunner(loader.getSegment(segment));
final List<ScanResultValue> 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<Object>) results.get(0).getEvents());
for (ScanResultValue value : results) {
Assert.assertEquals(
hash,
HashBasedNumberedShardSpec.hash(getObjectMapper(), (List<Object>) value.getEvents())
);
}
}
}
}
private static class TestSupervisorTask extends TestParallelIndexSupervisorTask
{
TestSupervisorTask(
String id,
TaskResource taskResource,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> 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<PartialSegmentMergeIOConfig> 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<SubTaskSpec<PartialSegmentGenerateTask>> subTaskSpecIterator() throws IOException
{
final Iterator<SubTaskSpec<PartialSegmentGenerateTask>> iterator = super.subTaskSpecIterator();
return new Iterator<SubTaskSpec<PartialSegmentGenerateTask>>()
{
@Override
public boolean hasNext()
{
return iterator.hasNext();
}
@Override
public SubTaskSpec<PartialSegmentGenerateTask> next()
{
try {
Thread.sleep(10);
return iterator.next();
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
};
}
@Override
SubTaskSpec<PartialSegmentGenerateTask> 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<PartialSegmentGenerateTask>(
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<PartialSegmentMergeIOConfig> mergeIOConfigs,
IndexingServiceClient indexingServiceClient
)
{
super(
toolbox,
supervisorTask.getId(),
supervisorTask.getGroupId(),
supervisorTask.getIngestionSchema().getDataSchema(),
mergeIOConfigs,
supervisorTask.getIngestionSchema().getTuningConfig(),
supervisorTask.getContext(),
indexingServiceClient
);
this.supervisorTask = supervisorTask;
}
@Override
Iterator<SubTaskSpec<PartialSegmentMergeTask>> subTaskSpecIterator()
{
final Iterator<SubTaskSpec<PartialSegmentMergeTask>> iterator = super.subTaskSpecIterator();
return new Iterator<SubTaskSpec<PartialSegmentMergeTask>>()
{
@Override
public boolean hasNext()
{
return iterator.hasNext();
}
@Override
public SubTaskSpec<PartialSegmentMergeTask> next()
{
try {
Thread.sleep(10);
return iterator.next();
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
};
}
@Override
SubTaskSpec<PartialSegmentMergeTask> newTaskSpec(PartialSegmentMergeIOConfig ioConfig)
{
final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec(
supervisorTask.getIngestionSchema().getDataSchema(),
ioConfig,
getTuningConfig()
);
return new SubTaskSpec<PartialSegmentMergeTask>(
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<String, Object> context,
IndexingServiceClient indexingServiceClient,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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;
}
}
}

View File

@ -29,6 +29,7 @@ import org.apache.druid.data.input.impl.LocalInputSource;
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.ChatHandlerProvider;
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 @@ import java.util.Map;
public class ParallelIndexSupervisorTaskSerdeTest
{
private static final ObjectMapper OBJECT_MAPPER = createObjectMapper();
private static final List<Interval> INTERVALS = Collections.singletonList(Intervals.of("2018/2019"));
private static ObjectMapper createObjectMapper()
{
@ -83,7 +86,7 @@ public class ParallelIndexSupervisorTaskSerdeTest
ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTaskBuilder()
.ingestionSpec(
new ParallelIndexIngestionSpecBuilder()
.inputIntervals(Collections.singletonList(Intervals.of("2018/2019")))
.inputIntervals(INTERVALS)
.build()
)
.build();
@ -112,11 +115,11 @@ public class ParallelIndexSupervisorTaskSerdeTest
}
@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 class ParallelIndexSupervisorTaskSerdeTest
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 @@ public class ParallelIndexSupervisorTaskSerdeTest
@Nullable
PartitionsSpec partitionsSpec = null;
@SuppressWarnings("SameParameterValue")
ParallelIndexIngestionSpecBuilder inputIntervals(List<Interval> inputIntervals)
{
this.inputIntervals = inputIntervals;

View File

@ -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<String, Object> CONTEXT = Collections.emptyMap();
static final IndexingServiceClient INDEXING_SERVICE_CLIENT = TestUtils.INDEXING_SERVICE_CLIENT;
static final IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> 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<Interval> 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<Interval> 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<ParallelIndexSupervisorTaskClient> 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);
}
}
}

View File

@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.indexing.common.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));
}
}

View File

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

View File

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

View File

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

View File

@ -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<Interval> granularitySpecInputIntervals = Collections.singletonList(Intervals.ETERNITY);
private boolean forceGuaranteedRollup = true;
private PartitionsSpec partitionsSpec = PARTITIONS_SPEC;
PerfectRollupWorkerTaskBuilder granularitySpecInputIntervals(List<Interval> 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<Interval> 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<String, Object> 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();
}
}
}

View File

@ -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<InputRow> 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<InputRow> 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<InputRow> nullInputRowIterator =
IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(null);
GranularitySpec absentBucketIntervalGranularitySpec =
IndexTaskInputRowIteratorBuilderTestingFactory.createAbsentBucketIntervalGranularitySpec(invalidTimestamp);
List<IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler> 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<InputRow> 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<InputRow> inputRowIterator =
IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow);
GranularitySpec absentBucketIntervalGranularitySpec =
IndexTaskInputRowIteratorBuilderTestingFactory.createAbsentBucketIntervalGranularitySpec(timestamp);
List<IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler> 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<InputRow> inputRowIterator =
IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow);
GranularitySpec granularitySpec = IndexTaskInputRowIteratorBuilderTestingFactory.createGranularitySpec(
timestamp,
IndexTaskInputRowIteratorBuilderTestingFactory.PRESENT_BUCKET_INTERVAL_OPT
);
List<HandlingInputRowIterator.InputRowHandler> appendedHandlers = Collections.singletonList(row -> true);
List<IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler> 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<InputRow> inputRowIterator =
IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(inputRow);
GranularitySpec granularitySpec = IndexTaskInputRowIteratorBuilderTestingFactory.createGranularitySpec(
timestamp,
IndexTaskInputRowIteratorBuilderTestingFactory.PRESENT_BUCKET_INTERVAL_OPT
);
List<IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler> handlerInvocationHistory =
HANDLER_TESTER.invokeHandlers(inputRowIterator, granularitySpec, inputRow);
Assert.assertEquals(Collections.emptyList(), handlerInvocationHistory);
}
}
}

View File

@ -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<Interval> 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<String> 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<InputRow> createInputRowIterator(InputRow inputRow)
{
return new CloseableIterator<InputRow>()
{
@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<Interval> bucketIntervalOpt)
{
GranularitySpec granularitySpec = EasyMock.mock(GranularitySpec.class);
EasyMock.expect(granularitySpec.bucketInterval(timestamp)).andStubReturn(bucketIntervalOpt);
EasyMock.replay(granularitySpec);
return granularitySpec;
}
static HandlerTester createHandlerTester(Supplier<IndexTaskInputRowIteratorBuilder> iteratorBuilderSupplier)
{
return new HandlerTester(iteratorBuilderSupplier);
}
static class HandlerTester
{
enum Handler
{
NULL_ROW,
ABSENT_BUCKET_INTERVAL,
APPENDED
}
private final Supplier<IndexTaskInputRowIteratorBuilder> iteratorBuilderSupplier;
private HandlerTester(Supplier<IndexTaskInputRowIteratorBuilder> iteratorBuilderSupplier)
{
this.iteratorBuilderSupplier = iteratorBuilderSupplier;
}
List<Handler> invokeHandlers(
CloseableIterator<InputRow> inputRowIterator,
GranularitySpec granularitySpec,
InputRow expectedNextInputRow
)
{
return invokeHandlers(
inputRowIterator,
granularitySpec,
Collections.emptyList(),
expectedNextInputRow
);
}
List<Handler> invokeHandlers(
CloseableIterator<InputRow> inputRowIterator,
GranularitySpec granularitySpec,
List<HandlingInputRowIterator.InputRowHandler> appendedHandlers,
InputRow expectedNextInputRow
)
{
List<Handler> 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;
}
}
}

View File

@ -21,8 +21,8 @@ package org.apache.druid.tests.indexer;
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 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
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();

View File

@ -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<String, String> rollupTransform = spec -> {
try {
spec = StringUtils.replace(

View File

@ -49,6 +49,7 @@ class JodaStuff
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<Period>) periodDeserializer);
@ -76,6 +77,15 @@ class JodaStuff
}
}
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

View File

@ -39,6 +39,8 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
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 class TestHelper
}
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);
}
}
}

View File

@ -21,6 +21,7 @@ package org.apache.druid.segment.realtime.firehose;
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<StringInputR
{
private final String data;
@VisibleForTesting
@JsonCreator
public InlineFirehoseFactory(@JsonProperty("data") String data)
{