Parallel indexing single dim partitions (#8925)

* Parallel indexing single dim partitions

Implements single dimension range partitioning for native parallel batch
indexing as described in #8769. This initial version requires the
druid-datasketches extension to be loaded.

The algorithm has 5 phases that are orchestrated by the supervisor in
`ParallelIndexSupervisorTask#runRangePartitionMultiPhaseParallel()`.
These phases and the main classes involved are described below:

1) In parallel, determine the distribution of dimension values for each
   input source split.

   `PartialDimensionDistributionTask` uses `StringSketch` to generate
   the approximate distribution of dimension values for each input
   source split. If the rows are ungrouped,
   `PartialDimensionDistributionTask.UngroupedRowDimensionValueFilter`
   uses a Bloom filter to skip rows that would be grouped. The final
   distribution is sent back to the supervisor via
   `DimensionDistributionReport`.

2) The range partitions are determined.

   In `ParallelIndexSupervisorTask#determineAllRangePartitions()`, the
   supervisor uses `StringSketchMerger` to merge the individual
   `StringSketch`es created in the preceding phase. The merged sketch is
   then used to create the range partitions.

3) In parallel, generate partial range-partitioned segments.

   `PartialRangeSegmentGenerateTask` uses the range partitions
   determined in the preceding phase and
   `RangePartitionCachingLocalSegmentAllocator` to generate
   `SingleDimensionShardSpec`s.  The partition information is sent back
   to the supervisor via `GeneratedGenericPartitionsReport`.

4) The partial range segments are grouped.

   In `ParallelIndexSupervisorTask#groupGenericPartitionLocationsPerPartition()`,
   the supervisor creates the `PartialGenericSegmentMergeIOConfig`s
   necessary for the next phase.

5) In parallel, merge partial range-partitioned segments.

   `PartialGenericSegmentMergeTask` uses `GenericPartitionLocation` to
   retrieve the partial range-partitioned segments generated earlier and
   then merges and publishes them.

* Fix dependencies & forbidden apis

* Fixes for integration test

* Address review comments

* Fix docs, strict compile, sketch check, rollup check

* Fix first shard spec, partition serde, single subtask

* Fix first partition check in test

* Misc rewording/refactoring to address code review

* Fix doc link

* Split batch index integration test

* Do not run parallel-batch-index twice

* Adjust last partition

* Split ITParallelIndexTest to reduce runtime

* Rename test class

* Allow null values in range partitions

* Indicate which phase failed

* Improve asserts in tests
This commit is contained in:
Chi Cao Minh 2019-12-09 23:05:49 -08:00 committed by Gian Merlino
parent a6dcc99962
commit bab78fc80e
67 changed files with 5554 additions and 113 deletions

View File

@ -290,6 +290,13 @@ jobs:
docker exec -it druid-$v sh -c 'dmesg | tail -3' ;
done
- &integration_perfect_rollup_parallel_batch_index
name: "perfect rollup parallel batch index integration test"
services: *integration_test_services
env: TESTNG_GROUPS='-Dgroups=perfect-rollup-parallel-batch-index'
script: *run_integration_test
after_failure: *integration_test_diags
- &integration_kafka_index
name: "kafka index integration test"
services: *integration_test_services
@ -314,6 +321,6 @@ jobs:
- &integration_tests
name: "other integration test"
services: *integration_test_services
env: TESTNG_GROUPS='-DexcludedGroups=batch-index,kafka-index,query,realtime-index'
env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index'
script: *run_integration_test
after_failure: *integration_test_diags

View File

@ -175,7 +175,11 @@ public class SingleDimensionPartitionsSpec implements DimensionBasedPartitionsSp
@Override
public String getForceGuaranteedRollupIncompatiblityReason()
{
return NAME + " partitions unsupported";
if (getPartitionDimension() == null) {
return PARITION_DIMENSION + " must be specified";
}
return FORCE_GUARANTEED_ROLLUP_COMPATIBLE;
}
@Override

View File

@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.ISE;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* {@link ShardSpec} for range partitioning based on a single dimension
@ -184,4 +185,26 @@ public class SingleDimensionShardSpec implements ShardSpec
", partitionNum=" + partitionNum +
'}';
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SingleDimensionShardSpec that = (SingleDimensionShardSpec) o;
return partitionNum == that.partitionNum &&
Objects.equals(dimension, that.dimension) &&
Objects.equals(start, that.start) &&
Objects.equals(end, that.end);
}
@Override
public int hashCode()
{
return Objects.hash(dimension, start, end, partitionNum);
}
}

View File

@ -366,7 +366,7 @@ The configuration options are:
|type|Type of partitionSpec to be used.|"single_dim"|
|targetRowsPerSegment|Target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|yes|
|targetPartitionSize|Deprecated. Renamed to `targetRowsPerSegment`. Target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|no|
|maxRowsPerSegment|Maximum number of rows to include in a partition. Defaults to 50% larger than the `targetPartitionSize`.|no|
|maxRowsPerSegment|Maximum number of rows to include in a partition. Defaults to 50% larger than the `targetRowsPerSegment`.|no|
|maxPartitionSize|Deprecated. Use `maxRowsPerSegment` instead. Maximum number of rows to include in a partition. Defaults to 50% larger than the `targetPartitionSize`.|no|
|partitionDimension|The dimension to partition on. Leave blank to select a dimension automatically.|no|
|assumeGrouped|Assume that input data has already been grouped on time and dimensions. Ingestion will run faster, but may choose sub-optimal partitions if this assumption is violated.|no|

View File

@ -88,7 +88,7 @@ This table compares the three available options:
| **Input locations** | Any [firehose](native-batch.md#firehoses). | Any [firehose](native-batch.md#firehoses). | Any Hadoop FileSystem or Druid datasource. |
| **File formats** | Text file formats (CSV, TSV, JSON). Support for binary formats is coming in a future release. | Text file formats (CSV, TSV, JSON). Support for binary formats is coming in a future release. | Any Hadoop InputFormat. |
| **[Rollup modes](#rollup)** | Perfect if `forceGuaranteedRollup` = true in the [`tuningConfig`](native-batch.md#tuningconfig).| Perfect if `forceGuaranteedRollup` = true in the [`tuningConfig`](native-batch.md#tuningconfig). | Always perfect. |
| **Partitioning options** | Hash-based partitioning is supported when `forceGuaranteedRollup` = true in the [`tuningConfig`](native-batch.md#tuningconfig). | Hash-based partitioning (when `forceGuaranteedRollup` = true). | Hash-based or range-based partitioning via [`partitionsSpec`](hadoop.md#partitionsspec). |
| **Partitioning options** | Hash-based partitioning is supported when `forceGuaranteedRollup` = true in the [`tuningConfig`](native-batch.md#tuningconfig). | Hash-based or range-based partitioning (when `forceGuaranteedRollup` = true). | Hash-based or range-based partitioning via [`partitionsSpec`](hadoop.md#partitionsspec). |
<a name="data-model"></a>

View File

@ -54,7 +54,7 @@ each sub task creates segments individually and reports them to the supervisor t
If `forceGuaranteedRollup` = true, it's executed in two phases with data shuffle which is similar to [MapReduce](https://en.wikipedia.org/wiki/MapReduce).
In the first phase, each sub task partitions input data based on `segmentGranularity` (primary partition key) in `granularitySpec`
and `partitionDimensions` (secondary partition key) in `partitionsSpec`. The partitioned data is served by
and `partitionDimension` or `partitionDimensions` (secondary partition key) in `partitionsSpec`. The partitioned data is served by
the [middleManager](../design/middlemanager.md) or the [indexer](../design/indexer.md)
where the first phase tasks ran. In the second phase, each sub task fetches
partitioned data from MiddleManagers or indexers and merges them to create the final segments.
@ -205,13 +205,13 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
|maxTotalRows|Deprecated. Use `partitionsSpec` instead. Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|20000000|no|
|numShards|Deprecated. Use `partitionsSpec` instead. Directly specify the number of shards to create. If this is specified and `intervals` is specified in the `granularitySpec`, the index task can skip the determine intervals/partitions pass through the data. `numShards` cannot be specified if `maxRowsPerSegment` is set.|null|no|
|numShards|Deprecated. Use `partitionsSpec` instead. Directly specify the number of shards to create when using a `hashed` `partitionsSpec`. If this is specified and `intervals` is specified in the `granularitySpec`, the index task can skip the determine intervals/partitions pass through the data. `numShards` cannot be specified if `maxRowsPerSegment` is set.|null|no|
|splitHintSpec|Used to give a hint to control the amount of data that each first phase task reads. This hint could be ignored depending on the implementation of firehose. See [SplitHintSpec](#splithintspec) for more details.|null|no|
|partitionsSpec|Defines how to partition data in each timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic` if `forceGuaranteedRollup` = false, `hashed` if `forceGuaranteedRollup` = true|no|
|partitionsSpec|Defines how to partition data in each timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic` if `forceGuaranteedRollup` = false, `hashed` or `single_dim` if `forceGuaranteedRollup` = true|no|
|indexSpec|Defines segment storage format options to be used at indexing time, see [IndexSpec](index.md#indexspec)|null|no|
|indexSpecForIntermediatePersists|Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. this can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. however, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](index.md#indexspec) for possible values.|same as indexSpec|no|
|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../ingestion/index.md#rollup). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. If this is set to true, `numShards` in `tuningConfig` and `intervals` in `granularitySpec` must be set. Note that the result segments would be hash-partitioned. This flag cannot be used with `appendToExisting` of IOConfig. For more details, see the below __Segment pushing modes__ section.|false|no|
|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../ingestion/index.md#rollup). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. If this is set to true, `intervals` in `granularitySpec` must be set and `hashed` or `single_dim` must be used for `partitionsSpec`. This flag cannot be used with `appendToExisting` of IOConfig. For more details, see the below __Segment pushing modes__ section.|false|no|
|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no|
|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [SegmentWriteOutMediumFactory](#segmentwriteoutmediumfactory).|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` is used|no|
@ -241,18 +241,43 @@ Currently only one splitHintSpec, i.e., `segments`, is available.
### `partitionsSpec`
PartitionsSpec is to describe the secondary partitioning method.
PartitionsSpec is used to describe the secondary partitioning method.
You should use different partitionsSpec depending on the [rollup mode](../ingestion/index.md#rollup) you want.
For perfect rollup, you should use `hashed`.
For perfect rollup, you should use either `hashed` (partitioning based on the hash of dimensions in each row) or
`single_dim` (based on ranges of a single dimension). For best-effort rollup, you should use `dynamic`.
The three `partitionsSpec` types have different pros and cons:
- `dynamic`: Fastest ingestion speed. Guarantees a well-balanced distribution in segment size. Only best-effort rollup.
- `hashed`: Moderate ingestion speed. Creates a well-balanced distribution in segment size. Allows perfect rollup.
- `single_dim`: Slowest ingestion speed. Segment sizes may be skewed depending on the partition key, but the broker can
use the partition information to efficiently prune segments early to speed up queries. Allows perfect rollup.
#### Hash-based partitioning
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|This should always be `hashed`|none|yes|
|targetRowsPerSegment|Target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|5000000 (if `numShards` is not set)|either this or `numShards`|
|numShards|Directly specify the number of shards to create. If this is specified and `intervals` is specified in the `granularitySpec`, the index task can skip the determine intervals/partitions pass through the data. `numShards` cannot be specified if `targetRowsPerSegment` is set.|null|no|
|partitionDimensions|The dimensions to partition on. Leave blank to select all dimensions. Only used with `numShards`, will be ignored when `targetRowsPerSegment` is set.|null|no|
|numShards|Directly specify the number of shards to create. If this is specified and `intervals` is specified in the `granularitySpec`, the index task can skip the determine intervals/partitions pass through the data. `numShards` cannot be specified if `targetRowsPerSegment` is set.|null|yes|
|partitionDimensions|The dimensions to partition on. Leave blank to select all dimensions.|null|no|
For best-effort rollup, you should use `dynamic`.
#### Single-dimension range partitioning
> Single-dimension range partitioning currently requires the
> [druid-datasketches](../development/extensions-core/datasketches-extension.md)
> extension to be [loaded from the classpath](../development/extensions.md#loading-extensions-from-the-classpath).
> Because single-range partitioning makes two passes over the input, the index task may fail if the input changes
> in between the two passes.
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|This should always be `single_dim`|none|yes|
|partitionDimension|The dimension to partition on. Only rows with a single dimension value will be included.|none|yes|
|targetRowsPerSegment|Target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|none|either this or `maxRowsPerSegment`|
|maxRowsPerSegment|Maximum number of rows to include in a partition. Defaults to 50% larger than the `targetRowsPerSegment`.|none|either this or `targetRowsPerSegment`|
|assumeGrouped|Assume that input data has already been grouped on time and dimensions. Ingestion will run faster, but may choose sub-optimal partitions if this assumption is violated.|false|no|
#### Dynamic partitioning
|property|description|default|required?|
|--------|-----------|-------|---------|
@ -943,4 +968,4 @@ A spec that applies a filter and reads a subset of the original datasource's col
}
```
This spec above will only return the `page`, `user` dimensions and `added` metric. Only rows where `page` = `Druid` will be returned.
This spec above will only return the `page`, `user` dimensions and `added` metric. Only rows where `page` = `Druid` will be returned.

View File

@ -34,16 +34,10 @@
<relativePath>../../pom.xml</relativePath>
</parent>
<properties>
<datasketches.core.version>1.1.0-incubating</datasketches.core.version>
<datasketches.memory.version>1.2.0-incubating</datasketches.memory.version>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-java</artifactId>
<version>${datasketches.core.version}</version>
<exclusions>
<exclusion>
<groupId>com.google.code.findbugs</groupId>
@ -54,7 +48,6 @@
<dependency>
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-memory</artifactId>
<version>${datasketches.memory.version}</version>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>

View File

@ -199,6 +199,26 @@
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
<dependency>
<!-- Used in native parallel batch indexing to determine distribution of dimension values -->
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-java</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<!-- Used in native parallel batch indexing to determine distribution of dimension values -->
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-memory</artifactId>
<scope>provided</scope>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -23,6 +23,7 @@ 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.SurrogateAction;
import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
@ -43,8 +44,9 @@ import java.util.stream.Collectors;
* Allocates all necessary segments locally at the beginning and reuses them.
*
* @see HashPartitionCachingLocalSegmentAllocator
* @see RangePartitionCachingLocalSegmentAllocator
*/
class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
{
private final String taskId;
private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
@ -55,27 +57,30 @@ class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
{
/**
* @param versionFinder Returns the version for the specified interval
*
* @return Information for segment preallocation
*/
Map<Interval, List<SegmentIdWithShardSpec>> create(Function<Interval, String> versionFinder);
}
CachingLocalSegmentAllocator(
CachingLocalSegmentAllocatorHelper(
TaskToolbox toolbox,
String taskId,
String supervisorTaskId,
IntervalToSegmentIdsCreator intervalToSegmentIdsCreator
) throws IOException
{
this.taskId = taskId;
this.sequenceNameToSegmentId = new HashMap<>();
final Map<Interval, String> intervalToVersion = toolbox.getTaskActionClient()
.submit(new LockListAction())
.stream()
.collect(Collectors.toMap(
TaskLock::getInterval,
TaskLock::getVersion
));
final Map<Interval, String> intervalToVersion =
toolbox.getTaskActionClient()
.submit(new SurrogateAction<>(supervisorTaskId, 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);

View File

@ -39,7 +39,7 @@ import java.util.stream.IntStream;
/**
* Allocates all necessary hash-partitioned segments locally at the beginning and reuses them.
*
* @see CachingLocalSegmentAllocator
* @see CachingLocalSegmentAllocatorHelper
*/
public class HashPartitionCachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
{
@ -51,6 +51,7 @@ public class HashPartitionCachingLocalSegmentAllocator implements IndexTaskSegme
public HashPartitionCachingLocalSegmentAllocator(
TaskToolbox toolbox,
String taskId,
String supervisorTaskId,
String dataSource,
Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec
) throws IOException
@ -59,9 +60,10 @@ public class HashPartitionCachingLocalSegmentAllocator implements IndexTaskSegme
this.dataSource = dataSource;
this.allocateSpec = allocateSpec;
this.delegate = new CachingLocalSegmentAllocator(
this.delegate = new CachingLocalSegmentAllocatorHelper(
toolbox,
taskId,
supervisorTaskId,
this::getIntervalToSegmentIds
);
}

View File

@ -828,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 HashPartitionCachingLocalSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec);
return new HashPartitionCachingLocalSegmentAllocator(toolbox, getId(), getId(), getDataSource(), allocateSpec);
} else {
return new LocalSegmentAllocator(toolbox, getId(), getDataSource(), dataSchema.getGranularitySpec());
}

View File

@ -0,0 +1,152 @@
/*
* 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.collect.Maps;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
* Allocates all necessary range-partitioned segments locally at the beginning and reuses them.
*
* @see CachingLocalSegmentAllocatorHelper
*/
public class RangePartitionCachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
{
private final String dataSource;
private final String partitionDimension;
private final Map<Interval, PartitionBoundaries> intervalsToPartitions;
private final IndexTaskSegmentAllocator delegate;
public RangePartitionCachingLocalSegmentAllocator(
TaskToolbox toolbox,
String taskId,
String supervisorTaskId,
String dataSource,
String partitionDimension,
Map<Interval, PartitionBoundaries> intervalsToPartitions
) throws IOException
{
this.dataSource = dataSource;
this.partitionDimension = partitionDimension;
this.intervalsToPartitions = intervalsToPartitions;
this.delegate = new CachingLocalSegmentAllocatorHelper(
toolbox,
taskId,
supervisorTaskId,
this::getIntervalToSegmentIds
);
}
private Map<Interval, List<SegmentIdWithShardSpec>> getIntervalToSegmentIds(Function<Interval, String> versionFinder)
{
Map<Interval, List<SegmentIdWithShardSpec>> intervalToSegmentIds =
Maps.newHashMapWithExpectedSize(intervalsToPartitions.size());
intervalsToPartitions.forEach(
(interval, partitionBoundaries) ->
intervalToSegmentIds.put(
interval,
translatePartitionBoundaries(interval, partitionBoundaries, versionFinder)
)
);
return intervalToSegmentIds;
}
/**
* Translate {@link PartitionBoundaries} into the corresponding
* {@link org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec} with segment id.
*/
private List<SegmentIdWithShardSpec> translatePartitionBoundaries(
Interval interval,
PartitionBoundaries partitionBoundaries,
Function<Interval, String> versionFinder
)
{
if (partitionBoundaries.isEmpty()) {
return Collections.emptyList();
}
return IntStream.range(0, partitionBoundaries.size() - 1)
.mapToObj(i -> createSegmentIdWithShardSpec(
interval,
versionFinder.apply(interval),
partitionBoundaries.get(i),
partitionBoundaries.get(i + 1),
i
))
.collect(Collectors.toList());
}
private SegmentIdWithShardSpec createSegmentIdWithShardSpec(
Interval interval,
String version,
String partitionStart,
@Nullable String partitionEnd,
int partitionNum
)
{
// The shardSpec created here will be reused in PartialGenericSegmentMergeTask. This is ok because
// all PartialSegmentGenerateTasks create the same set of segmentIds (and thus shardSpecs).
return new SegmentIdWithShardSpec(
dataSource,
interval,
version,
new SingleDimensionShardSpec(
partitionDimension,
partitionStart,
partitionEnd,
partitionNum
)
);
}
@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

@ -28,8 +28,11 @@ 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.PartialDimensionDistributionTask;
import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask;
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.PartialRangeSegmentGenerateTask;
import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunner;
@ -60,6 +63,9 @@ import java.util.Map;
@Type(name = SinglePhaseSubTask.OLD_TYPE_NAME, value = LegacySinglePhaseSubTask.class), // for backward compatibility
@Type(name = PartialHashSegmentGenerateTask.TYPE, value = PartialHashSegmentGenerateTask.class),
@Type(name = PartialHashSegmentMergeTask.TYPE, value = PartialHashSegmentMergeTask.class),
@Type(name = PartialRangeSegmentGenerateTask.TYPE, value = PartialRangeSegmentGenerateTask.class),
@Type(name = PartialDimensionDistributionTask.TYPE, value = PartialDimensionDistributionTask.class),
@Type(name = PartialGenericSegmentMergeTask.TYPE, value = PartialGenericSegmentMergeTask.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,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.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution;
import org.joda.time.Interval;
import java.util.Map;
public class DimensionDistributionReport implements SubTaskReport
{
static final String TYPE = "dimension_distribution";
private static final String PROP_DISTRIBUTIONS = "distributions";
private final String taskId;
private final Map<Interval, StringDistribution> intervalToDistribution;
@JsonCreator
public DimensionDistributionReport(
@JsonProperty("taskId") String taskId,
@JsonProperty(PROP_DISTRIBUTIONS) Map<Interval, StringDistribution> intervalToDistribution
)
{
this.taskId = taskId;
this.intervalToDistribution = intervalToDistribution;
}
@Override
@JsonProperty
public String getTaskId()
{
return taskId;
}
@JsonProperty(PROP_DISTRIBUTIONS)
public Map<Interval, StringDistribution> getIntervalToDistribution()
{
return intervalToDistribution;
}
@Override
public String toString()
{
return "DimensionDistributionReport{" +
"taskId='" + taskId + '\'' +
", intervalToDistribution=" + intervalToDistribution +
'}';
}
}

View File

@ -0,0 +1,43 @@
/*
* 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 GenericPartitionStat}s created by a {@link PartialSegmentGenerateTask}. This report is
* collected by {@link ParallelIndexSupervisorTask} and used to generate {@link PartialGenericSegmentMergeIOConfig}.
*/
class GeneratedPartitionsMetadataReport extends GeneratedPartitionsReport<GenericPartitionStat> implements SubTaskReport
{
public static final String TYPE = "generated_partitions_metadata";
@JsonCreator
GeneratedPartitionsMetadataReport(
@JsonProperty("taskId") String taskId,
@JsonProperty("partitionStats") List<GenericPartitionStat> partitionStats
)
{
super(taskId, partitionStats);
}
}

View File

@ -27,7 +27,7 @@ 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 PartialHashSegmentMergeIOConfig}.
* used to generate {@link PartialSegmentMergeIOConfig}.
*/
abstract class GeneratedPartitionsReport<T extends PartitionStat> implements SubTaskReport
{

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.timeline.partition.ShardSpec;
import org.joda.time.Interval;
/**
* This class represents the intermediary data server where the partition of {@link #interval} and {@link #shardSpec}
* is stored.
*/
public class GenericPartitionLocation extends PartitionLocation<ShardSpec>
{
private final ShardSpec shardSpec;
@JsonCreator
public GenericPartitionLocation(
@JsonProperty("host") String host,
@JsonProperty("port") int port,
@JsonProperty("useHttps") boolean useHttps,
@JsonProperty("subTaskId") String subTaskId,
@JsonProperty("interval") Interval interval,
@JsonProperty("shardSpec") ShardSpec shardSpec
)
{
super(host, port, useHttps, subTaskId, interval, shardSpec);
this.shardSpec = shardSpec;
}
@JsonIgnore
@Override
public int getPartitionId()
{
return shardSpec.getPartitionNum();
}
@JsonProperty
ShardSpec getShardSpec()
{
return shardSpec;
}
}

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.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.timeline.partition.ShardSpec;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Objects;
/**
* Generic partition description ({@link ShardSpec}) and statistics created by {@link PartialSegmentGenerateTask}. Each
* partition is a set of data of the same time chunk (primary partition key) and the same {@link ShardSpec} (secondary
* partition key). The {@link ShardSpec} is later used by {@link PartialGenericSegmentMergeTask} to merge the partial
* segments.
*/
public class GenericPartitionStat extends PartitionStat<ShardSpec>
{
private static final String PROP_SHARD_SPEC = "shardSpec";
// Secondary partition key
private final ShardSpec shardSpec;
@JsonCreator
public GenericPartitionStat(
@JsonProperty("taskExecutorHost") String taskExecutorHost,
@JsonProperty("taskExecutorPort") int taskExecutorPort,
@JsonProperty("useHttps") boolean useHttps,
@JsonProperty("interval") Interval interval,
@JsonProperty(PROP_SHARD_SPEC) ShardSpec shardSpec,
@JsonProperty("numRows") @Nullable Integer numRows,
@JsonProperty("sizeBytes") @Nullable Long sizeBytes
)
{
super(taskExecutorHost, taskExecutorPort, useHttps, interval, numRows, sizeBytes);
this.shardSpec = shardSpec;
}
@Override
public int getPartitionId()
{
return shardSpec.getPartitionNum();
}
@JsonProperty(PROP_SHARD_SPEC)
@Override
ShardSpec getSecondaryPartition()
{
return shardSpec;
}
@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;
}
GenericPartitionStat that = (GenericPartitionStat) o;
return Objects.equals(shardSpec, that.shardSpec);
}
@Override
public int hashCode()
{
return Objects.hash(super.hashCode(), shardSpec);
}
}

View File

@ -27,6 +27,8 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.Multimap;
import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputFormat;
@ -34,8 +36,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.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
import org.apache.druid.indexing.common.Counters;
import org.apache.druid.indexing.common.TaskLock;
@ -56,6 +58,11 @@ import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistributionMerger;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketchMerger;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
@ -75,6 +82,7 @@ import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.utils.CollectionUtils;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -93,6 +101,7 @@ import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -298,6 +307,36 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
);
}
@VisibleForTesting
PartialDimensionDistributionParallelIndexTaskRunner createPartialDimensionDistributionRunner(TaskToolbox toolbox)
{
return new PartialDimensionDistributionParallelIndexTaskRunner(
toolbox,
getId(),
getGroupId(),
ingestionSchema,
getContext(),
indexingServiceClient
);
}
@VisibleForTesting
PartialRangeSegmentGenerateParallelIndexTaskRunner createPartialRangeSegmentGenerateRunner(
TaskToolbox toolbox,
Map<Interval, PartitionBoundaries> intervalToPartitions
)
{
return new PartialRangeSegmentGenerateParallelIndexTaskRunner(
toolbox,
getId(),
getGroupId(),
ingestionSchema,
getContext(),
indexingServiceClient,
intervalToPartitions
);
}
@VisibleForTesting
PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner(
TaskToolbox toolbox,
@ -316,12 +355,53 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
);
}
@VisibleForTesting
PartialGenericSegmentMergeParallelIndexTaskRunner createPartialGenericSegmentMergeRunner(
TaskToolbox toolbox,
List<PartialGenericSegmentMergeIOConfig> ioConfigs
)
{
return new PartialGenericSegmentMergeParallelIndexTaskRunner(
toolbox,
getId(),
getGroupId(),
getIngestionSchema().getDataSchema(),
ioConfigs,
getIngestionSchema().getTuningConfig(),
getContext(),
indexingServiceClient
);
}
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
if (useRangePartitions()) {
assertDataSketchesAvailable();
}
return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.getDataSchema().getGranularitySpec());
}
private boolean useRangePartitions()
{
return (ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec() instanceof SingleDimensionPartitionsSpec);
}
private static void assertDataSketchesAvailable()
{
try {
//noinspection ResultOfObjectAllocationIgnored
new StringSketch();
}
catch (NoClassDefFoundError e) {
throw new ISE(
e,
"DataSketches is unvailable."
+ " Try loading the druid-datasketches extension from the classpath for the overlord and middleManagers/indexers."
);
}
}
@Override
public List<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> intervals)
throws IOException
@ -436,7 +516,11 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
private boolean isParallelMode()
{
return baseInputSource.isSplittable() && ingestionSchema.getTuningConfig().getMaxNumConcurrentSubTasks() > 1;
// Range partitioning is not implemented for runSequential() (but hash partitioning is)
int minRequiredNumConcurrentSubTasks = useRangePartitions() ? 1 : 2;
return baseInputSource.isSplittable()
&& ingestionSchema.getTuningConfig().getMaxNumConcurrentSubTasks() >= minRequiredNumConcurrentSubTasks;
}
/**
@ -471,16 +555,9 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
*/
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);
return useRangePartitions()
? runRangePartitionMultiPhaseParallel(toolbox)
: runHashPartitionMultiPhaseParallel(toolbox);
}
private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throws Exception
@ -519,6 +596,88 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
return TaskStatus.fromCode(getId(), state);
}
private TaskStatus runRangePartitionMultiPhaseParallel(TaskToolbox toolbox) throws Exception
{
ParallelIndexTaskRunner<PartialDimensionDistributionTask, DimensionDistributionReport> distributionRunner =
createRunner(
toolbox,
this::createPartialDimensionDistributionRunner
);
TaskState distributionState = runNextPhase(distributionRunner);
if (distributionState.isFailure()) {
return TaskStatus.failure(getId(), PartialDimensionDistributionTask.TYPE + " failed");
}
Map<Interval, PartitionBoundaries> intervalToPartitions =
determineAllRangePartitions(distributionRunner.getReports().values());
if (intervalToPartitions.isEmpty()) {
String msg = "No valid rows for single dimension partitioning."
+ " All rows may have invalid timestamps or multiple dimension values.";
LOG.warn(msg);
return TaskStatus.success(getId(), msg);
}
ParallelIndexTaskRunner<PartialRangeSegmentGenerateTask, GeneratedPartitionsReport<GenericPartitionStat>> indexingRunner =
createRunner(toolbox, tb -> createPartialRangeSegmentGenerateRunner(tb, intervalToPartitions));
TaskState indexingState = runNextPhase(indexingRunner);
if (indexingState.isFailure()) {
return TaskStatus.failure(getId(), PartialRangeSegmentGenerateTask.TYPE + " failed");
}
// partition (interval, partitionId) -> partition locations
Map<Pair<Interval, Integer>, List<GenericPartitionLocation>> partitionToLocations =
groupGenericPartitionLocationsPerPartition(indexingRunner.getReports());
final List<PartialGenericSegmentMergeIOConfig> ioConfigs = createGenericMergeIOConfigs(
ingestionSchema.getTuningConfig().getTotalNumMergeTasks(),
partitionToLocations
);
ParallelIndexTaskRunner<PartialGenericSegmentMergeTask, PushedSegmentsReport> mergeRunner = createRunner(
toolbox,
tb -> createPartialGenericSegmentMergeRunner(tb, ioConfigs)
);
TaskState mergeState = runNextPhase(mergeRunner);
if (mergeState.isSuccess()) {
publishSegments(toolbox, mergeRunner.getReports());
}
return TaskStatus.fromCode(getId(), mergeState);
}
private Map<Interval, PartitionBoundaries> determineAllRangePartitions(Collection<DimensionDistributionReport> reports)
{
Multimap<Interval, StringDistribution> intervalToDistributions = ArrayListMultimap.create();
reports.forEach(report -> {
Map<Interval, StringDistribution> intervalToDistribution = report.getIntervalToDistribution();
intervalToDistribution.forEach(intervalToDistributions::put);
});
return CollectionUtils.mapValues(intervalToDistributions.asMap(), this::determineRangePartition);
}
private PartitionBoundaries determineRangePartition(Collection<StringDistribution> distributions)
{
StringDistributionMerger distributionMerger = new StringSketchMerger();
distributions.forEach(distributionMerger::merge);
StringDistribution mergedDistribution = distributionMerger.getResult();
SingleDimensionPartitionsSpec partitionsSpec =
(SingleDimensionPartitionsSpec) ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec();
final PartitionBoundaries partitions;
Integer targetRowsPerSegment = partitionsSpec.getTargetRowsPerSegment();
if (targetRowsPerSegment == null) {
partitions = mergedDistribution.getEvenPartitionsByMaxSize(partitionsSpec.getMaxRowsPerSegment());
} else {
partitions = mergedDistribution.getEvenPartitionsByTargetSize(targetRowsPerSegment);
}
return partitions;
}
private static Map<Pair<Interval, Integer>, List<HashPartitionLocation>> groupHashPartitionLocationsPerPartition(
Map<String, GeneratedHashPartitionsReport> subTaskIdToReport
)
@ -537,6 +696,24 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
return groupPartitionLocationsPerPartition(subTaskIdToReport, createPartitionLocationFunction);
}
private static Map<Pair<Interval, Integer>, List<GenericPartitionLocation>> groupGenericPartitionLocationsPerPartition(
Map<String, GeneratedPartitionsReport<GenericPartitionStat>> subTaskIdToReport
)
{
BiFunction<String, GenericPartitionStat, GenericPartitionLocation> createPartitionLocationFunction =
(subtaskId, partitionStat) ->
new GenericPartitionLocation(
partitionStat.getTaskExecutorHost(),
partitionStat.getTaskExecutorPort(),
partitionStat.isUseHttps(),
subtaskId,
partitionStat.getInterval(),
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,
@ -572,6 +749,18 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
);
}
private static List<PartialGenericSegmentMergeIOConfig> createGenericMergeIOConfigs(
int totalNumMergeTasks,
Map<Pair<Interval, Integer>, List<GenericPartitionLocation>> partitionToLocations
)
{
return createMergeIOConfigs(
totalNumMergeTasks,
partitionToLocations,
PartialGenericSegmentMergeIOConfig::new
);
}
private static <M extends PartialSegmentMergeIOConfig, L extends PartitionLocation> List<M> createMergeIOConfigs(
int totalNumMergeTasks,
Map<Pair<Interval, Integer>, List<L>> partitionToLocations,

View File

@ -0,0 +1,124 @@
/*
* 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 java.util.Map;
/**
* {@link ParallelIndexTaskRunner} for the phase to determine distribution of dimension values in
* multi-phase parallel indexing.
*/
class PartialDimensionDistributionParallelIndexTaskRunner
extends InputSourceSplitParallelIndexTaskRunner<PartialDimensionDistributionTask, DimensionDistributionReport>
{
// For tests
private final IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory;
PartialDimensionDistributionParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
)
{
this(
toolbox,
taskId,
groupId,
ingestionSchema,
context,
indexingServiceClient,
null
);
}
@VisibleForTesting
PartialDimensionDistributionParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory
)
{
super(
toolbox,
taskId,
groupId,
ingestionSchema,
context,
indexingServiceClient
);
this.taskClientFactory = taskClientFactory;
}
@Override
public String getName()
{
return PartialDimensionDistributionTask.TYPE;
}
@Override
SubTaskSpec<PartialDimensionDistributionTask> createSubTaskSpec(
String id,
String groupId,
String supervisorTaskId,
Map<String, Object> context,
InputSplit split,
ParallelIndexIngestionSpec subTaskIngestionSpec,
IndexingServiceClient indexingServiceClient
)
{
return new SubTaskSpec<PartialDimensionDistributionTask>(
id,
groupId,
supervisorTaskId,
context,
split
)
{
@Override
public PartialDimensionDistributionTask newSubTask(int numAttempts)
{
return new PartialDimensionDistributionTask(
null,
getGroupId(),
null,
getSupervisorTaskId(),
numAttempts,
subTaskIngestionSpec,
getContext(),
getIndexingServiceClient(),
taskClientFactory
);
}
};
}
}

View File

@ -0,0 +1,474 @@
/*
* 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.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.hash.BloomFilter;
import org.apache.druid.client.indexing.IndexingServiceClient;
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;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSourceReader;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.TimeDimTuple;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.TimeDimTupleFactory;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.TimeDimTupleFunnel;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.RangePartitionIndexTaskInputRowIteratorBuilder;
import org.apache.druid.java.util.common.granularity.Granularity;
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;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import java.util.stream.Collectors;
/**
* The worker task of {@link PartialDimensionDistributionParallelIndexTaskRunner}. This task
* determines the distribution of dimension values of input data.
*/
public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
{
public static final String TYPE = "partial_dimension_distribution";
private static final Logger LOG = new Logger(PartialDimensionDistributionTask.class);
// Future work: StringDistribution does not handle inserting NULLs. This is the same behavior as hadoop indexing.
private static final boolean SKIP_NULL = true;
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
private final String supervisorTaskId;
private final IndexingServiceClient indexingServiceClient;
private final IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory;
// For testing
private final Supplier<DedupRowDimensionValueFilter> dedupRowDimValueFilterSupplier;
@JsonCreator
PartialDimensionDistributionTask(
// 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
)
{
this(
id,
groupId,
taskResource,
supervisorTaskId,
numAttempts,
ingestionSchema,
context,
indexingServiceClient,
taskClientFactory,
() -> new DedupRowDimensionValueFilter(
ingestionSchema.getDataSchema().getGranularitySpec().getQueryGranularity()
)
);
}
@VisibleForTesting // Only for testing
PartialDimensionDistributionTask(
@Nullable String id,
final String groupId,
final TaskResource taskResource,
final String supervisorTaskId,
final int numAttempts,
final ParallelIndexIngestionSpec ingestionSchema,
final Map<String, Object> context,
IndexingServiceClient indexingServiceClient,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory,
Supplier<DedupRowDimensionValueFilter> dedupRowDimValueFilterSupplier
)
{
super(
getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()),
groupId,
taskResource,
ingestionSchema.getDataSchema(),
ingestionSchema.getTuningConfig(),
context
);
Preconditions.checkArgument(
ingestionSchema.getTuningConfig().getPartitionsSpec() instanceof SingleDimensionPartitionsSpec,
"%s partitionsSpec required",
SingleDimensionPartitionsSpec.NAME
);
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.indexingServiceClient = indexingServiceClient;
this.taskClientFactory = taskClientFactory;
this.dedupRowDimValueFilterSupplier = dedupRowDimValueFilterSupplier;
}
@JsonProperty
private int getNumAttempts()
{
return numAttempts;
}
@JsonProperty("spec")
private ParallelIndexIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@JsonProperty
private 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
public TaskStatus runTask(TaskToolbox toolbox) throws Exception
{
DataSchema dataSchema = ingestionSchema.getDataSchema();
GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
SingleDimensionPartitionsSpec partitionsSpec = (SingleDimensionPartitionsSpec) tuningConfig.getPartitionsSpec();
Preconditions.checkNotNull(partitionsSpec, "partitionsSpec required in tuningConfig");
String partitionDimension = partitionsSpec.getPartitionDimension();
Preconditions.checkNotNull(partitionDimension, "partitionDimension required in partitionsSpec");
boolean isAssumeGrouped = partitionsSpec.isAssumeGrouped();
InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
List<String> metricsNames = Arrays.stream(dataSchema.getAggregators())
.map(AggregatorFactory::getName)
.collect(Collectors.toList());
InputFormat inputFormat = inputSource.needsFormat()
? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema)
: null;
InputSourceReader inputSourceReader = dataSchema.getTransformSpec().decorate(
inputSource.reader(
new InputRowSchema(
dataSchema.getTimestampSpec(),
dataSchema.getDimensionsSpec(),
metricsNames
),
inputFormat,
toolbox.getIndexingTmpDir()
)
);
try (
CloseableIterator<InputRow> inputRowIterator = inputSourceReader.read();
HandlingInputRowIterator iterator = new RangePartitionIndexTaskInputRowIteratorBuilder(partitionDimension, SKIP_NULL)
.delegate(inputRowIterator)
.granularitySpec(granularitySpec)
.nullRowRunnable(IndexTaskInputRowIteratorBuilder.NOOP_RUNNABLE)
.absentBucketIntervalConsumer(IndexTaskInputRowIteratorBuilder.NOOP_CONSUMER)
.build()
) {
Map<Interval, StringDistribution> distribution = determineDistribution(
iterator,
granularitySpec,
partitionDimension,
isAssumeGrouped,
tuningConfig.isLogParseExceptions(),
tuningConfig.getMaxParseExceptions()
);
sendReport(new DimensionDistributionReport(getId(), distribution));
}
return TaskStatus.success(getId());
}
private Map<Interval, StringDistribution> determineDistribution(
HandlingInputRowIterator inputRowIterator,
GranularitySpec granularitySpec,
String partitionDimension,
boolean isAssumeGrouped,
boolean isLogParseExceptions,
int maxParseExceptions
)
{
Map<Interval, StringDistribution> intervalToDistribution = new HashMap<>();
DimensionValueFilter dimValueFilter =
!isAssumeGrouped && granularitySpec.isRollup()
? dedupRowDimValueFilterSupplier.get()
: new PassthroughRowDimensionValueFilter();
int numParseExceptions = 0;
while (inputRowIterator.hasNext()) {
try {
InputRow inputRow = inputRowIterator.next();
if (inputRow == null) {
continue;
}
DateTime timestamp = inputRow.getTimestamp();
//noinspection OptionalGetWithoutIsPresent (InputRowIterator returns rows with present intervals)
Interval interval = granularitySpec.bucketInterval(timestamp).get();
StringDistribution stringDistribution =
intervalToDistribution.computeIfAbsent(interval, k -> new StringSketch());
String dimensionValue = dimValueFilter.accept(
interval,
timestamp,
Iterables.getOnlyElement(inputRow.getDimension(partitionDimension))
);
if (dimensionValue != null) {
stringDistribution.put(dimensionValue);
}
}
catch (ParseException e) {
if (isLogParseExceptions) {
LOG.error(e, "Encountered parse exception:");
}
numParseExceptions++;
if (numParseExceptions > maxParseExceptions) {
throw new RuntimeException("Max parse exceptions exceeded, terminating task...");
}
}
}
// DedupRowDimensionValueFilter may not accept the min/max dimensionValue. If needed, add the min/max
// values to the distributions so they have an accurate min/max.
dimValueFilter.getIntervalToMinDimensionValue()
.forEach((interval, min) -> intervalToDistribution.get(interval).putIfNewMin(min));
dimValueFilter.getIntervalToMaxDimensionValue()
.forEach((interval, max) -> intervalToDistribution.get(interval).putIfNewMax(max));
return intervalToDistribution;
}
private void sendReport(DimensionDistributionReport report)
{
final ParallelIndexSupervisorTaskClient taskClient = taskClientFactory.build(
new ClientBasedTaskInfoProvider(indexingServiceClient),
getId(),
1, // always use a single http thread
ingestionSchema.getTuningConfig().getChatHandlerTimeout(),
ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
);
taskClient.report(supervisorTaskId, report);
}
private interface DimensionValueFilter
{
/**
* @return Dimension value if it should be accepted, else null
*/
@Nullable
String accept(Interval interval, DateTime timestamp, String dimensionValue);
/**
* @return Minimum dimension value for each interval processed so far.
*/
Map<Interval, String> getIntervalToMinDimensionValue();
/**
* @return Maximum dimension value for each interval processed so far.
*/
Map<Interval, String> getIntervalToMaxDimensionValue();
}
/**
* Filters out reoccurrences of rows that have timestamps with the same query granularity and dimension value.
* Approximate matching is used, so there is a small probability that rows that are not reoccurences are discarded.
*/
@VisibleForTesting
static class DedupRowDimensionValueFilter implements DimensionValueFilter
{
// A bloom filter is used to approximately group rows by query granularity. These values assume
// time chunks have fewer than BLOOM_FILTER_EXPECTED_INSERTIONS rows. With the below values, the
// Bloom filter will use about 170MB of memory.
//
// For more details on the Bloom filter memory consumption:
// https://github.com/google/guava/issues/2520#issuecomment-231233736
private static final int BLOOM_FILTER_EXPECTED_INSERTIONS = 100_000_000;
private static final double BLOOM_FILTER_EXPECTED_FALSE_POSITIVE_PROBABILTY = 0.001;
private final PassthroughRowDimensionValueFilter delegate;
private final TimeDimTupleFactory timeDimTupleFactory;
private final BloomFilter<TimeDimTuple> timeDimTupleBloomFilter;
DedupRowDimensionValueFilter(Granularity queryGranularity)
{
this(queryGranularity, BLOOM_FILTER_EXPECTED_INSERTIONS, BLOOM_FILTER_EXPECTED_FALSE_POSITIVE_PROBABILTY);
}
@VisibleForTesting // to allow controlling false positive rate of bloom filter
DedupRowDimensionValueFilter(
Granularity queryGranularity,
int bloomFilterExpectedInsertions,
double bloomFilterFalsePositiveProbability
)
{
delegate = new PassthroughRowDimensionValueFilter();
timeDimTupleFactory = new TimeDimTupleFactory(queryGranularity);
timeDimTupleBloomFilter = BloomFilter.create(
TimeDimTupleFunnel.INSTANCE,
bloomFilterExpectedInsertions,
bloomFilterFalsePositiveProbability
);
}
@Nullable
@Override
public String accept(Interval interval, DateTime timestamp, String dimensionValue)
{
delegate.accept(interval, timestamp, dimensionValue);
TimeDimTuple timeDimTuple = timeDimTupleFactory.createWithBucketedTimestamp(timestamp, dimensionValue);
if (timeDimTupleBloomFilter.mightContain(timeDimTuple)) {
return null;
} else {
timeDimTupleBloomFilter.put(timeDimTuple);
return dimensionValue;
}
}
@Override
public Map<Interval, String> getIntervalToMinDimensionValue()
{
return delegate.getIntervalToMinDimensionValue();
}
@Override
public Map<Interval, String> getIntervalToMaxDimensionValue()
{
return delegate.getIntervalToMaxDimensionValue();
}
}
/**
* Accepts all input rows, even if they are reoccurrences of timestamps with the same query granularity and dimension
* value.
*/
private static class PassthroughRowDimensionValueFilter implements DimensionValueFilter
{
private final Map<Interval, String> intervalToMinDimensionValue;
private final Map<Interval, String> intervalToMaxDimensionValue;
PassthroughRowDimensionValueFilter()
{
this.intervalToMinDimensionValue = new HashMap<>();
this.intervalToMaxDimensionValue = new HashMap<>();
}
@Override
@Nullable
public String accept(Interval interval, DateTime timestamp, String dimensionValue)
{
updateMinDimensionValue(interval, dimensionValue);
updateMaxDimensionValue(interval, dimensionValue);
return dimensionValue;
}
private void updateMinDimensionValue(Interval interval, String dimensionValue)
{
intervalToMinDimensionValue.compute(
interval,
(intervalKey, currentMinValue) -> {
if (currentMinValue == null || dimensionValue.compareTo(currentMinValue) < 0) {
return dimensionValue;
} else {
return currentMinValue;
}
}
);
}
private void updateMaxDimensionValue(Interval interval, String dimensionValue)
{
intervalToMaxDimensionValue.compute(
interval,
(intervalKey, currentMaxValue) -> {
if (currentMaxValue == null || dimensionValue.compareTo(currentMaxValue) > 0) {
return dimensionValue;
} else {
return currentMaxValue;
}
}
);
}
@Override
public Map<Interval, String> getIntervalToMinDimensionValue()
{
return intervalToMinDimensionValue;
}
@Override
public Map<Interval, String> getIntervalToMaxDimensionValue()
{
return intervalToMaxDimensionValue;
}
}
}

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(PartialGenericSegmentMergeTask.TYPE)
class PartialGenericSegmentMergeIOConfig extends PartialSegmentMergeIOConfig<GenericPartitionLocation>
implements IOConfig
{
@JsonCreator
PartialGenericSegmentMergeIOConfig(
@JsonProperty("partitionLocations") List<GenericPartitionLocation> partitionLocations
)
{
super(partitionLocations);
}
}

View File

@ -0,0 +1,38 @@
/*
* 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 PartialGenericSegmentMergeIngestionSpec
extends PartialSegmentMergeIngestionSpec<PartialGenericSegmentMergeIOConfig>
{
@JsonCreator
PartialGenericSegmentMergeIngestionSpec(
@JsonProperty("dataSchema") DataSchema dataSchema,
@JsonProperty("ioConfig") PartialGenericSegmentMergeIOConfig ioConfig,
@JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig
)
{
super(dataSchema, ioConfig, tuningConfig);
}
}

View File

@ -0,0 +1,110 @@
/*
* 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.segment.indexing.DataSchema;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
* {@link ParallelIndexTaskRunner} for the phase to merge partitioned segments in multi-phase parallel indexing.
*/
class PartialGenericSegmentMergeParallelIndexTaskRunner
extends ParallelIndexPhaseRunner<PartialGenericSegmentMergeTask, PushedSegmentsReport>
{
private final DataSchema dataSchema;
private final List<PartialGenericSegmentMergeIOConfig> mergeIOConfigs;
PartialGenericSegmentMergeParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
DataSchema dataSchema,
List<PartialGenericSegmentMergeIOConfig> mergeIOConfigs,
ParallelIndexTuningConfig tuningConfig,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
)
{
super(toolbox, taskId, groupId, tuningConfig, context, indexingServiceClient);
this.dataSchema = dataSchema;
this.mergeIOConfigs = mergeIOConfigs;
}
@Override
public String getName()
{
return PartialGenericSegmentMergeTask.TYPE;
}
@Override
Iterator<SubTaskSpec<PartialGenericSegmentMergeTask>> subTaskSpecIterator()
{
return mergeIOConfigs.stream().map(this::newTaskSpec).iterator();
}
@Override
int estimateTotalNumSubTasks()
{
return mergeIOConfigs.size();
}
@VisibleForTesting
SubTaskSpec<PartialGenericSegmentMergeTask> newTaskSpec(PartialGenericSegmentMergeIOConfig ioConfig)
{
final PartialGenericSegmentMergeIngestionSpec ingestionSpec = new PartialGenericSegmentMergeIngestionSpec(
dataSchema,
ioConfig,
getTuningConfig()
);
return new SubTaskSpec<PartialGenericSegmentMergeTask>(
getTaskId() + "_" + getAndIncrementNextSpecId(),
getGroupId(),
getTaskId(),
getContext(),
new InputSplit<>(ioConfig.getPartitionLocations())
)
{
@Override
public PartialGenericSegmentMergeTask newSubTask(int numAttempts)
{
return new PartialGenericSegmentMergeTask(
null,
getGroupId(),
null,
getSupervisorTaskId(),
numAttempts,
ingestionSpec,
getContext(),
null,
null,
null
);
}
};
}
}

View File

@ -0,0 +1,134 @@
/*
* 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 com.google.common.collect.HashBasedTable;
import com.google.common.collect.Table;
import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.guice.annotations.EscalatedClient;
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.ShardSpec;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
/**
* {@link ParallelIndexTaskRunner} for the phase to merge generic partitioned segments in multi-phase parallel indexing.
*/
public class PartialGenericSegmentMergeTask extends PartialSegmentMergeTask<ShardSpec, GenericPartitionLocation>
{
public static final String TYPE = "partial_index_generic_merge";
private final PartialGenericSegmentMergeIngestionSpec ingestionSchema;
private final Table<Interval, Integer, ShardSpec> intervalAndIntegerToShardSpec;
@JsonCreator
public PartialGenericSegmentMergeTask(
// 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 PartialGenericSegmentMergeIngestionSpec 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;
this.intervalAndIntegerToShardSpec = createIntervalAndIntegerToShardSpec(
ingestionSchema.getIOConfig().getPartitionLocations()
);
}
private static Table<Interval, Integer, ShardSpec> createIntervalAndIntegerToShardSpec(
List<GenericPartitionLocation> partitionLocations
)
{
Table<Interval, Integer, ShardSpec> intervalAndIntegerToShardSpec = HashBasedTable.create();
partitionLocations.forEach(
p -> {
ShardSpec currShardSpec = intervalAndIntegerToShardSpec.get(p.getInterval(), p.getPartitionId());
Preconditions.checkArgument(
currShardSpec == null || p.getShardSpec().equals(currShardSpec),
"interval %s, partitionId %s mismatched shard specs: %s",
p.getInterval(),
p.getPartitionId(),
partitionLocations
);
intervalAndIntegerToShardSpec.put(p.getInterval(), p.getPartitionId(), p.getShardSpec());
}
);
return intervalAndIntegerToShardSpec;
}
@JsonProperty("spec")
private PartialGenericSegmentMergeIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@Override
public String getType()
{
return TYPE;
}
@Override
ShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId)
{
return Preconditions.checkNotNull(
intervalAndIntegerToShardSpec.get(interval, partitionId),
"no shard spec exists for interval %s, partitionId %s: %s",
interval,
partitionId,
intervalAndIntegerToShardSpec.rowMap()
);
}
}

View File

@ -130,6 +130,7 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
return new HashPartitionCachingLocalSegmentAllocator(
toolbox,
getId(),
supervisorTaskId,
getDataSource(),
createShardSpecs()
);

View File

@ -102,10 +102,10 @@ public class PartialHashSegmentMergeTask
}
@Override
HashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionNum)
HashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId)
{
return new HashBasedNumberedShardSpec(
partitionNum,
partitionId,
Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"),
partitionsSpec.getPartitionDimensions(),
toolbox.getJsonMapper()

View File

@ -0,0 +1,131 @@
/*
* 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.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.joda.time.Interval;
import java.util.Map;
/**
* {@link ParallelIndexTaskRunner} for the phase to create range partitioned segments in multi-phase parallel indexing.
*
* @see PartialHashSegmentMergeParallelIndexTaskRunner
*/
class PartialRangeSegmentGenerateParallelIndexTaskRunner
extends InputSourceSplitParallelIndexTaskRunner<PartialRangeSegmentGenerateTask, GeneratedPartitionsReport<GenericPartitionStat>>
{
private final IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory;
private final AppenderatorsManager appenderatorsManager;
private final Map<Interval, PartitionBoundaries> intervalToPartitions;
PartialRangeSegmentGenerateParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient,
Map<Interval, PartitionBoundaries> intervalToPartitions
)
{
this(
toolbox,
taskId,
groupId,
ingestionSchema,
context,
indexingServiceClient,
intervalToPartitions,
null,
null
);
}
@VisibleForTesting
PartialRangeSegmentGenerateParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient,
Map<Interval, PartitionBoundaries> intervalToPartitions,
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory,
AppenderatorsManager appenderatorsManager
)
{
super(toolbox, taskId, groupId, ingestionSchema, context, indexingServiceClient);
this.taskClientFactory = taskClientFactory;
this.appenderatorsManager = appenderatorsManager;
this.intervalToPartitions = intervalToPartitions;
}
@Override
public String getName()
{
return PartialRangeSegmentGenerateTask.TYPE;
}
@Override
SubTaskSpec<PartialRangeSegmentGenerateTask> createSubTaskSpec(
String id,
String groupId,
String supervisorTaskId,
Map<String, Object> context,
InputSplit split,
ParallelIndexIngestionSpec subTaskIngestionSpec,
IndexingServiceClient indexingServiceClient
)
{
return new SubTaskSpec<PartialRangeSegmentGenerateTask>(
id,
groupId,
supervisorTaskId,
context,
split
)
{
@Override
public PartialRangeSegmentGenerateTask newSubTask(int numAttempts)
{
return new PartialRangeSegmentGenerateTask(
null,
groupId,
null,
supervisorTaskId,
numAttempts,
subTaskIngestionSpec,
context,
intervalToPartitions,
indexingServiceClient,
taskClientFactory,
appenderatorsManager
);
}
};
}
}

View File

@ -0,0 +1,185 @@
/*
* 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.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator;
import org.apache.druid.indexing.common.task.RangePartitionCachingLocalSegmentAllocator;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.RangePartitionIndexTaskInputRowIteratorBuilder;
import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.timeline.DataSegment;
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 PartialRangeSegmentGenerateParallelIndexTaskRunner}. This task partitions input data by
* ranges of the partition dimension specified in {@link SingleDimensionPartitionsSpec}. Partitioned segments are stored
* in local storage using {@link ShuffleDataSegmentPusher}.
*/
public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<GeneratedPartitionsMetadataReport>
{
public static final String TYPE = "partial_range_index_generate";
private static final String PROP_SPEC = "spec";
private static final boolean SKIP_NULL = true;
private final String supervisorTaskId;
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
private final Map<Interval, PartitionBoundaries> intervalToPartitions;
@JsonCreator
public PartialRangeSegmentGenerateTask(
// id shouldn't be null except when this task is created by ParallelIndexSupervisorTask
@JsonProperty("id") @Nullable String id,
@JsonProperty("groupId") String groupId,
@JsonProperty("resource") TaskResource taskResource,
@JsonProperty("supervisorTaskId") String supervisorTaskId,
@JsonProperty("numAttempts") int numAttempts, // zero-based counting
@JsonProperty(PROP_SPEC) ParallelIndexIngestionSpec ingestionSchema,
@JsonProperty("context") Map<String, Object> context,
@JsonProperty("intervalToPartitions") Map<Interval, PartitionBoundaries> intervalToPartitions,
@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 RangePartitionIndexTaskInputRowIteratorBuilder(getPartitionDimension(ingestionSchema), !SKIP_NULL)
);
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.intervalToPartitions = intervalToPartitions;
}
private static String getPartitionDimension(ParallelIndexIngestionSpec ingestionSpec)
{
PartitionsSpec partitionsSpec = ingestionSpec.getTuningConfig().getPartitionsSpec();
Preconditions.checkArgument(
partitionsSpec instanceof SingleDimensionPartitionsSpec,
"%s partitionsSpec required",
SingleDimensionPartitionsSpec.NAME
);
SingleDimensionPartitionsSpec singleDimPartitionsSpec = (SingleDimensionPartitionsSpec) partitionsSpec;
String partitionDimension = singleDimPartitionsSpec.getPartitionDimension();
Preconditions.checkNotNull(partitionDimension, "partitionDimension required");
return partitionDimension;
}
@JsonProperty
public int getNumAttempts()
{
return numAttempts;
}
@JsonProperty(PROP_SPEC)
public ParallelIndexIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@JsonProperty
public Map<Interval, PartitionBoundaries> getIntervalToPartitions()
{
return intervalToPartitions;
}
@Override
public String getType()
{
return TYPE;
}
@Override
public boolean isReady(TaskActionClient taskActionClient)
{
return true;
}
@Override
IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException
{
return new RangePartitionCachingLocalSegmentAllocator(
toolbox,
getId(),
supervisorTaskId,
getDataSource(),
getPartitionDimension(ingestionSchema),
intervalToPartitions
);
}
@Override
GeneratedPartitionsMetadataReport createGeneratedPartitionsReport(TaskToolbox toolbox, List<DataSegment> segments)
{
List<GenericPartitionStat> partitionStats = segments.stream()
.map(segment -> createPartitionStat(toolbox, segment))
.collect(Collectors.toList());
return new GeneratedPartitionsMetadataReport(getId(), partitionStats);
}
private GenericPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment)
{
return new GenericPartitionStat(
toolbox.getTaskExecutorNode().getHost(),
toolbox.getTaskExecutorNode().getPortToUse(),
toolbox.getTaskExecutorNode().isEnableTlsPort(),
segment.getInterval(),
segment.getShardSpec(),
null, // numRows is not supported yet
null // sizeBytes is not supported yet
);
}
}

View File

@ -281,7 +281,7 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec, P extends PartitionL
/**
* Create a {@link ShardSpec} suitable for the desired secondary partitioning strategy.
*/
abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int partitionNum);
abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId);
private Set<DataSegment> mergeAndPushSegments(
TaskToolbox toolbox,

View File

@ -30,7 +30,9 @@ 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 = GeneratedHashPartitionsReport.TYPE, value = GeneratedHashPartitionsReport.class)
@Type(name = GeneratedHashPartitionsReport.TYPE, value = GeneratedHashPartitionsReport.class),
@Type(name = DimensionDistributionReport.TYPE, value = DimensionDistributionReport.class),
@Type(name = GeneratedPartitionsMetadataReport.TYPE, value = GeneratedPartitionsMetadataReport.class)
})
public interface SubTaskReport
{

View File

@ -0,0 +1,77 @@
/*
* 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.distribution;
import com.google.common.collect.ForwardingList;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
/**
* List of range partition boundaries.
*/
public class PartitionBoundaries extends ForwardingList<String> implements List<String>
{
private final List<String> delegate;
// For jackson
@SuppressWarnings("unused")
private PartitionBoundaries()
{
delegate = new ArrayList<>();
}
/**
* @param partitions Elements corresponding to evenly-spaced fractional ranks of the distribution
*/
public PartitionBoundaries(String... partitions)
{
if (partitions.length == 0) {
delegate = Collections.emptyList();
return;
}
// Future improvement: Handle skewed partitions better (e.g., many values are repeated).
List<String> partitionBoundaries = Arrays.stream(partitions)
.distinct()
.collect(Collectors.toCollection(ArrayList::new));
// First partition starts with null (see StringPartitionChunk.isStart())
partitionBoundaries.set(0, null);
// Last partition ends with null (see StringPartitionChunk.isEnd())
if (partitionBoundaries.size() == 1) {
partitionBoundaries.add(null);
} else {
partitionBoundaries.set(partitionBoundaries.size() - 1, null);
}
delegate = Collections.unmodifiableList(partitionBoundaries);
}
@Override
protected List<String> delegate()
{
return delegate;
}
}

View File

@ -0,0 +1,66 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.indexing.common.task.batch.parallel.distribution;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
/**
* Counts frequencies of {@link String}s.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes({
@JsonSubTypes.Type(name = StringSketch.NAME, value = StringSketch.class)
})
public interface StringDistribution
{
/**
* Record occurrence of {@link String}
*/
void put(String element);
/**
* Record occurrence of {@link String} if it will become the new minimum element.
*/
void putIfNewMin(String element);
/**
* Record occurrence of {@link String} if it will become the new maximum element;
*/
void putIfNewMax(String element);
/**
* Split the distribution in the fewest number of evenly-sized partitions while honoring a max
* partition size.
*
* @return List of elements that correspond to the endpoints of evenly-sized partitions of the
* sorted elements.
*/
PartitionBoundaries getEvenPartitionsByMaxSize(int maxSize);
/**
* Split the distribution in the fewest number of evenly-sized partitions while honoring a target
* partition size (actual partition sizes may be slightly lower or higher).
*
* @return List of elements that correspond to the endpoints of evenly-sized partitions of the
* sorted elements.
*/
PartitionBoundaries getEvenPartitionsByTargetSize(int targetSize);
}

View File

@ -0,0 +1,36 @@
/*
* 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.distribution;
/**
* Merges {@link StringDistribution}s.
*/
public interface StringDistributionMerger
{
/**
* Merge distribution.
*/
void merge(StringDistribution distribution);
/**
* @return Merged distributions.
*/
StringDistribution getResult();
}

View File

@ -0,0 +1,207 @@
/*
* 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.distribution;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import com.fasterxml.jackson.databind.jsontype.TypeSerializer;
import com.fasterxml.jackson.databind.ser.std.StdSerializer;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.datasketches.ArrayOfStringsSerDe;
import org.apache.datasketches.memory.Memory;
import org.apache.datasketches.quantiles.ItemsSketch;
import java.io.IOException;
import java.util.Comparator;
/**
* Counts approximate frequencies of strings.
*/
@JsonSerialize(using = StringSketch.Jackson.Serializer.class)
@JsonDeserialize(using = StringSketch.Jackson.Deserializer.class)
public class StringSketch implements StringDistribution
{
static final String NAME = "sketch";
static final int SKETCH_K = 1 << 12; // smallest value with normalized rank error < 0.1%; retain up to ~86k elements
static final Comparator<String> SKETCH_COMPARATOR = Comparator.naturalOrder();
private static final ArrayOfStringsSerDe ARRAY_OF_STRINGS_SERDE = new ArrayOfStringsSerDe();
private final ItemsSketch<String> delegate;
public StringSketch()
{
this(ItemsSketch.getInstance(SKETCH_K, SKETCH_COMPARATOR));
}
StringSketch(ItemsSketch<String> sketch)
{
this.delegate = sketch;
}
@Override
public void put(String string)
{
delegate.update(string);
}
@Override
public void putIfNewMin(String string)
{
String min = delegate.getMinValue();
if (min == null || string.compareTo(min) < 0) {
delegate.update(string);
}
}
@Override
public void putIfNewMax(String string)
{
String max = delegate.getMaxValue();
if (max == null || string.compareTo(max) > 0) {
delegate.update(string);
}
}
@Override
public PartitionBoundaries getEvenPartitionsByMaxSize(int maxSize)
{
Preconditions.checkArgument(maxSize > 0, "maxSize must be positive but is %s", maxSize);
long n = delegate.getN();
double delta = delegate.getNormalizedRankError(true) * n; // account for approx distribution
int targetSize = Math.max(1, (int) Math.floor(maxSize - delta)); // floor() to increase chance below max size
int evenPartitionCount = (int) Math.ceil((double) n / targetSize); // ceil() to increase chance below max size
return getEvenPartitionsByCount(Math.max(1, evenPartitionCount));
}
@Override
public PartitionBoundaries getEvenPartitionsByTargetSize(int targetSize)
{
Preconditions.checkArgument(targetSize > 0, "targetSize must be positive but is %s", targetSize);
long n = delegate.getN();
int evenPartitionCount = Math.max(1, (int) Math.round((double) n / targetSize));
return getEvenPartitionsByCount(evenPartitionCount);
}
@VisibleForTesting
public String getMin()
{
return delegate.getMinValue();
}
@VisibleForTesting
public String getMax()
{
return delegate.getMaxValue();
}
private PartitionBoundaries getEvenPartitionsByCount(int evenPartitionCount)
{
Preconditions.checkArgument(
evenPartitionCount > 0,
"evenPartitionCount must be positive but is %s",
evenPartitionCount
);
String[] partitions = delegate.getQuantiles(evenPartitionCount + 1); // add 1 since this returns endpoints
return new PartitionBoundaries((partitions == null) ? new String[0] : partitions);
}
@Override
public String toString()
{
return "StringSketch{" +
"delegate=" + delegate +
'}';
}
ItemsSketch<String> getDelegate()
{
return delegate;
}
private byte[] toByteArray()
{
return delegate.toByteArray(ARRAY_OF_STRINGS_SERDE);
}
static class Jackson
{
private static final String FIELD_SKETCH = "sketch";
static class Serializer extends StdSerializer<StringSketch>
{
Serializer()
{
super(StringSketch.class);
}
@Override
public void serialize(
StringSketch stringSketch,
JsonGenerator jsonGenerator,
SerializerProvider serializerProvider
) throws IOException
{
jsonGenerator.writeBinaryField(FIELD_SKETCH, stringSketch.toByteArray());
}
@Override
public void serializeWithType(
StringSketch value,
JsonGenerator gen,
SerializerProvider serializers,
TypeSerializer typeSer
) throws IOException
{
typeSer.writeTypePrefixForObject(value, gen);
serialize(value, gen, serializers);
typeSer.writeTypeSuffixForObject(value, gen);
}
}
static class Deserializer extends StdDeserializer<StringSketch>
{
Deserializer()
{
super(StringSketch.class);
}
@Override
public StringSketch deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException
{
JsonNode jsonNode = jsonParser.getCodec().readTree(jsonParser);
byte[] sketchBytes = jsonNode.get(FIELD_SKETCH).binaryValue();
ItemsSketch<String> sketch = ItemsSketch.getInstance(
Memory.wrap(sketchBytes),
SKETCH_COMPARATOR,
ARRAY_OF_STRINGS_SERDE
);
return new StringSketch(sketch);
}
}
}
}

View File

@ -0,0 +1,52 @@
/*
* 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.distribution;
import org.apache.datasketches.quantiles.ItemsUnion;
/**
* Merges {@link StringSketch}es.
*/
public class StringSketchMerger implements StringDistributionMerger
{
private final ItemsUnion<String> delegate;
public StringSketchMerger()
{
delegate = ItemsUnion.getInstance(StringSketch.SKETCH_K, StringSketch.SKETCH_COMPARATOR);
}
@Override
public void merge(StringDistribution stringDistribution)
{
if (!(stringDistribution instanceof StringSketch)) {
throw new IllegalArgumentException("Only merging StringSketch instances is currently supported");
}
StringSketch stringSketch = (StringSketch) stringDistribution;
delegate.update(stringSketch.getDelegate());
}
@Override
public StringDistribution getResult()
{
return new StringSketch(delegate.getResult());
}
}

View File

@ -0,0 +1,86 @@
/*
* 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.distribution;
import java.util.Objects;
/**
* Tuple of timestamp and dimension value
*/
public class TimeDimTuple implements Comparable<TimeDimTuple>
{
private final long timestamp;
private final String dimensionValue;
TimeDimTuple(long timestamp, String dimensionValue)
{
this.timestamp = timestamp;
this.dimensionValue = dimensionValue;
}
public long getTimestamp()
{
return timestamp;
}
public String getDimensionValue()
{
return dimensionValue;
}
@Override
public int compareTo(TimeDimTuple o)
{
if (timestamp < o.timestamp) {
return -1;
}
if (o.timestamp < timestamp) {
return 1;
}
return dimensionValue.compareTo(o.dimensionValue);
}
@Override
public boolean equals(Object o)
{
if (!(o instanceof TimeDimTuple)) {
return false;
}
return compareTo((TimeDimTuple) o) == 0;
}
@Override
public int hashCode()
{
return Objects.hash(timestamp, dimensionValue);
}
@Override
public String toString()
{
return "TimeDimTuple{" +
"timestamp=" + timestamp +
", dimensionValue='" + dimensionValue + '\'' +
'}';
}
}

View File

@ -0,0 +1,47 @@
/*
* 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.distribution;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.joda.time.DateTime;
/**
* Creates {@link TimeDimTuple}s with time stamp adjust according to a {@link Granularity}.
*/
public class TimeDimTupleFactory
{
private final Granularity granularity;
public TimeDimTupleFactory(Granularity granularity)
{
this.granularity = granularity;
}
public TimeDimTuple createWithBucketedTimestamp(DateTime timestamp, String dimensionValue)
{
return new TimeDimTuple(getBucketTimestamp(timestamp), dimensionValue);
}
private long getBucketTimestamp(DateTime dateTime)
{
return granularity.bucketStart(dateTime).getMillis();
}
}

View File

@ -0,0 +1,38 @@
/*
* 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.distribution;
import com.google.common.hash.Funnel;
import com.google.common.hash.PrimitiveSink;
/**
* Utility class for adding {@link TimeDimTuple}s to a {@link com.google.common.hash.BloomFilter}.
*/
public enum TimeDimTupleFunnel implements Funnel<TimeDimTuple>
{
INSTANCE;
@Override
public void funnel(TimeDimTuple timeDimTuple, PrimitiveSink into)
{
into.putLong(timeDimTuple.getTimestamp())
.putUnencodedChars(timeDimTuple.getDimensionValue());
}
}

View File

@ -55,6 +55,8 @@ import java.util.function.Consumer;
* 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>
*
* @see RangePartitionIndexTaskInputRowIteratorBuilder
*/
public class DefaultIndexTaskInputRowIteratorBuilder implements IndexTaskInputRowIteratorBuilder
{

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.iterator;
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.parsers.CloseableIterator;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import java.util.List;
import java.util.function.Consumer;
/**
* <pre>
* Build an {@link HandlingInputRowIterator} for {@link IndexTask}s used for range partitioning. 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. Filter for rows with only a single dimension value count for the specified partition dimension.
*
* 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>
*
* @see DefaultIndexTaskInputRowIteratorBuilder
*/
public class RangePartitionIndexTaskInputRowIteratorBuilder implements IndexTaskInputRowIteratorBuilder
{
private final DefaultIndexTaskInputRowIteratorBuilder delegate;
/**
* @param partitionDimension Create range partitions for this dimension
* @param skipNull Whether to skip rows with a dimension value of null
*/
public RangePartitionIndexTaskInputRowIteratorBuilder(String partitionDimension, boolean skipNull)
{
delegate = new DefaultIndexTaskInputRowIteratorBuilder();
if (skipNull) {
delegate.appendInputRowHandler(createOnlySingleDimensionValueRowsHandler(partitionDimension));
} else {
delegate.appendInputRowHandler(createOnlySingleOrNullDimensionValueRowsHandler(partitionDimension));
}
}
@Override
public IndexTaskInputRowIteratorBuilder delegate(CloseableIterator<InputRow> inputRowIterator)
{
return delegate.delegate(inputRowIterator);
}
@Override
public IndexTaskInputRowIteratorBuilder granularitySpec(GranularitySpec granularitySpec)
{
return delegate.granularitySpec(granularitySpec);
}
@Override
public IndexTaskInputRowIteratorBuilder nullRowRunnable(Runnable nullRowRunnable)
{
return delegate.nullRowRunnable(nullRowRunnable);
}
@Override
public IndexTaskInputRowIteratorBuilder absentBucketIntervalConsumer(Consumer<InputRow> absentBucketIntervalConsumer)
{
return delegate.absentBucketIntervalConsumer(absentBucketIntervalConsumer);
}
@Override
public HandlingInputRowIterator build()
{
return delegate.build();
}
private static HandlingInputRowIterator.InputRowHandler createOnlySingleDimensionValueRowsHandler(
String partitionDimension
)
{
return inputRow -> {
List<String> dimensionValues = inputRow.getDimension(partitionDimension);
return dimensionValues.size() != 1;
};
}
private static HandlingInputRowIterator.InputRowHandler createOnlySingleOrNullDimensionValueRowsHandler(
String partitionDimension
)
{
return inputRow -> {
List<String> dimensionValues = inputRow.getDimension(partitionDimension);
return dimensionValues.size() > 1; // Rows.objectToStrings() returns an empty list for a single null value
};
}
}

View File

@ -0,0 +1,245 @@
/*
* 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.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.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class RangePartitionCachingLocalSegmentAllocatorTest
{
private static final String DATASOURCE = "datasource";
private static final String TASKID = "taskid";
private static final String SUPERVISOR_TASKID = "supervisor-taskid";
private static final String PARTITION_DIMENSION = "dimension";
private static final Interval INTERVAL_EMPTY = Intervals.utc(0, 1000);
private static final Interval INTERVAL_SINGLETON = Intervals.utc(1000, 2000);
private static final Interval INTERVAL_NORMAL = Intervals.utc(2000, 3000);
private static final Map<Interval, String> INTERVAL_TO_VERSION = ImmutableMap.of(
INTERVAL_EMPTY, "version-empty",
INTERVAL_SINGLETON, "version-singleton",
INTERVAL_NORMAL, "version-normal"
);
private static final String PARTITION0 = "0";
private static final String PARTITION5 = "5";
private static final String PARTITION9 = "9";
private static final PartitionBoundaries EMPTY_PARTITIONS = new PartitionBoundaries();
private static final PartitionBoundaries SINGLETON_PARTITIONS = new PartitionBoundaries(PARTITION0, PARTITION0);
private static final PartitionBoundaries NORMAL_PARTITIONS = new PartitionBoundaries(
PARTITION0,
PARTITION5,
PARTITION9
);
private static final Map<Interval, PartitionBoundaries> INTERVAL_TO_PARTITONS = ImmutableMap.of(
INTERVAL_EMPTY, EMPTY_PARTITIONS,
INTERVAL_SINGLETON, SINGLETON_PARTITIONS,
INTERVAL_NORMAL, NORMAL_PARTITIONS
);
private RangePartitionCachingLocalSegmentAllocator target;
@Rule
public ExpectedException exception = ExpectedException.none();
@Before
public void setup() throws IOException
{
TaskToolbox toolbox = createToolbox(
INTERVAL_TO_VERSION.keySet()
.stream()
.map(RangePartitionCachingLocalSegmentAllocatorTest::createTaskLock)
.collect(Collectors.toList())
);
target = new RangePartitionCachingLocalSegmentAllocator(
toolbox,
TASKID,
SUPERVISOR_TASKID,
DATASOURCE,
PARTITION_DIMENSION,
INTERVAL_TO_PARTITONS
);
}
@Test
public void failsIfAllocateFromEmptyInterval()
{
Interval interval = INTERVAL_EMPTY;
InputRow row = createInputRow(interval, PARTITION9);
exception.expect(IllegalStateException.class);
exception.expectMessage("Failed to get shardSpec");
String sequenceName = target.getSequenceName(interval, row);
allocate(row, sequenceName);
}
@Test
public void allocatesCorrectShardSpecsForSingletonPartitions()
{
Interval interval = INTERVAL_SINGLETON;
InputRow row = createInputRow(interval, PARTITION9);
testAllocate(row, interval, 0, null);
}
@Test
public void allocatesCorrectShardSpecsForFirstPartition()
{
Interval interval = INTERVAL_NORMAL;
InputRow row = createInputRow(interval, PARTITION0);
testAllocate(row, interval, 0);
}
@Test
public void allocatesCorrectShardSpecsForLastPartition()
{
Interval interval = INTERVAL_NORMAL;
InputRow row = createInputRow(interval, PARTITION9);
int partitionNum = INTERVAL_TO_PARTITONS.get(interval).size() - 2;
testAllocate(row, interval, partitionNum, null);
}
@SuppressWarnings("SameParameterValue")
private void testAllocate(InputRow row, Interval interval, int partitionNum)
{
String partitionEnd = getPartitionEnd(interval, partitionNum);
testAllocate(row, interval, partitionNum, partitionEnd);
}
@Nullable
private static String getPartitionEnd(Interval interval, int partitionNum)
{
PartitionBoundaries partitions = INTERVAL_TO_PARTITONS.get(interval);
boolean isLastPartition = (partitionNum + 1) == partitions.size();
return isLastPartition ? null : partitions.get(partitionNum + 1);
}
private void testAllocate(InputRow row, Interval interval, int partitionNum, @Nullable String partitionEnd)
{
String partitionStart = getPartitionStart(interval, partitionNum);
testAllocate(row, interval, partitionNum, partitionStart, partitionEnd);
}
@Nullable
private static String getPartitionStart(Interval interval, int partitionNum)
{
boolean isFirstPartition = partitionNum == 0;
return isFirstPartition ? null : INTERVAL_TO_PARTITONS.get(interval).get(partitionNum);
}
private void testAllocate(
InputRow row,
Interval interval,
int partitionNum,
@Nullable String partitionStart,
@Nullable String partitionEnd
)
{
String sequenceName = target.getSequenceName(interval, row);
SegmentIdWithShardSpec segmentIdWithShardSpec = allocate(row, sequenceName);
Assert.assertEquals(
SegmentId.of(DATASOURCE, interval, INTERVAL_TO_VERSION.get(interval), partitionNum),
segmentIdWithShardSpec.asSegmentId()
);
SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segmentIdWithShardSpec.getShardSpec();
Assert.assertEquals(PARTITION_DIMENSION, shardSpec.getDimension());
Assert.assertEquals(partitionNum, shardSpec.getPartitionNum());
Assert.assertEquals(partitionStart, shardSpec.getStart());
Assert.assertEquals(partitionEnd, shardSpec.getEnd());
}
private SegmentIdWithShardSpec allocate(InputRow row, String sequenceName)
{
try {
return target.allocate(row, sequenceName, null, false);
}
catch (IOException e) {
throw new UncheckedIOException(e);
}
}
private static TaskToolbox createToolbox(List<TaskLock> taskLocks)
{
TaskToolbox toolbox = EasyMock.mock(TaskToolbox.class);
EasyMock.expect(toolbox.getTaskActionClient()).andStubReturn(createTaskActionClient(taskLocks));
EasyMock.replay(toolbox);
return toolbox;
}
private static TaskActionClient createTaskActionClient(List<TaskLock> taskLocks)
{
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(Interval interval)
{
TaskLock taskLock = EasyMock.mock(TaskLock.class);
EasyMock.expect(taskLock.getInterval()).andStubReturn(interval);
EasyMock.expect(taskLock.getVersion()).andStubReturn(INTERVAL_TO_VERSION.get(interval));
EasyMock.replay(taskLock);
return taskLock;
}
private static InputRow createInputRow(Interval interval, String dimensionValue)
{
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(PARTITION_DIMENSION))
.andStubReturn(Collections.singletonList(dimensionValue));
EasyMock.replay(inputRow);
return inputRow;
}
}

View File

@ -0,0 +1,55 @@
/*
* 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.indexing.common.task.batch.parallel.distribution.StringDistribution;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.TestHelper;
import org.joda.time.Interval;
import org.junit.Before;
import org.junit.Test;
import java.util.Collections;
import java.util.Map;
public class DimensionDistributionReportTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
private DimensionDistributionReport target;
@Before
public void setup()
{
Interval interval = Intervals.ETERNITY;
StringSketch sketch = new StringSketch();
Map<Interval, StringDistribution> intervalToDistribution = Collections.singletonMap(interval, sketch);
String taskId = "abc";
target = new DimensionDistributionReport(taskId, intervalToDistribution);
}
@Test
public void serializesDeserializes()
{
TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target);
}
}

View File

@ -0,0 +1,58 @@
/*
* 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 GenericPartitionLocationTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
private GenericPartitionLocation target;
@Before
public void setup()
{
target = new GenericPartitionLocation(
ParallelIndexTestingFactory.HOST,
ParallelIndexTestingFactory.PORT,
ParallelIndexTestingFactory.USE_HTTPS,
ParallelIndexTestingFactory.SUBTASK_ID,
ParallelIndexTestingFactory.INTERVAL,
ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC
);
}
@Test
public void serializesDeserializes()
{
TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target);
}
@Test
public void hasPartitionIdThatMatchesShardSpec()
{
Assert.assertEquals(ParallelIndexTestingFactory.PARTITION_ID, target.getPartitionId());
}
}

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 GenericPartitionStatTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
private GenericPartitionStat target;
@Before
public void setup()
{
target = new GenericPartitionStat(
ParallelIndexTestingFactory.TASK_EXECUTOR_HOST,
ParallelIndexTestingFactory.TASK_EXECUTOR_PORT,
ParallelIndexTestingFactory.USE_HTTPS,
ParallelIndexTestingFactory.INTERVAL,
ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC,
ParallelIndexTestingFactory.NUM_ROWS,
ParallelIndexTestingFactory.SIZE_BYTES
);
}
@Test
public void serializesDeserializes()
{
TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target);
}
@Test
public void hasPartitionIdThatMatchesSecondaryPartition()
{
Assert.assertEquals(target.getSecondaryPartition().getPartitionNum(), target.getPartitionId());
}
}

View File

@ -53,6 +53,7 @@ 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 String SUPERVISOR_TASKID = "supervisor-taskid";
private static final Interval INTERVAL = Intervals.utc(0, 1000);
private static final String VERSION = "version";
private static final String DIMENSION = "dim";
@ -76,6 +77,7 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
target = new HashPartitionCachingLocalSegmentAllocator(
toolbox,
TASKID,
SUPERVISOR_TASKID,
DATASOURCE,
ALLOCATE_SPEC
);

View File

@ -153,14 +153,28 @@ public class ParallelIndexSupervisorTaskSerdeTest
}
@Test
public void forceGuaranteedRollupWithSingleDimPartitionsInvalid()
public void forceGuaranteedRollupWithSingleDimPartitionsMissingDimension()
{
expectedException.expect(IllegalStateException.class);
expectedException.expectMessage(
"forceGuaranteedRollup is incompatible with partitionsSpec: single_dim partitions unsupported"
"forceGuaranteedRollup is incompatible with partitionsSpec: partitionDimension must be specified"
);
new ParallelIndexSupervisorTaskBuilder()
.ingestionSpec(
new ParallelIndexIngestionSpecBuilder()
.forceGuaranteedRollup(true)
.partitionsSpec(new SingleDimensionPartitionsSpec(1, null, null, true))
.inputIntervals(INTERVALS)
.build()
)
.build();
}
@Test
public void forceGuaranteedRollupWithSingleDimPartitionsValid()
{
ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTaskBuilder()
.ingestionSpec(
new ParallelIndexIngestionSpecBuilder()
.forceGuaranteedRollup(true)
@ -169,6 +183,9 @@ public class ParallelIndexSupervisorTaskSerdeTest
.build()
)
.build();
PartitionsSpec partitionsSpec = task.getIngestionSchema().getTuningConfig().getPartitionsSpec();
Assert.assertThat(partitionsSpec, CoreMatchers.instanceOf(SingleDimensionPartitionsSpec.class));
}
private static class ParallelIndexSupervisorTaskBuilder

View File

@ -27,6 +27,7 @@ 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.JsonInputFormat;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
@ -44,6 +45,7 @@ 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.easymock.EasyMock;
import org.joda.time.Duration;
import org.joda.time.Interval;
@ -229,7 +231,14 @@ class ParallelIndexTestingFactory
static IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> createTaskClientFactory()
{
return TASK_CLIENT_FACTORY;
return (taskInfoProvider, callerId, numThreads, httpTimeout, numRetries) -> createTaskClient();
}
private static ParallelIndexSupervisorTaskClient createTaskClient()
{
ParallelIndexSupervisorTaskClient taskClient = EasyMock.niceMock(ParallelIndexSupervisorTaskClient.class);
EasyMock.replay(taskClient);
return taskClient;
}
static String createRow(long timestamp, Object dimensionValue)
@ -244,4 +253,9 @@ class ParallelIndexTestingFactory
throw new RuntimeException(e);
}
}
static InputFormat getInputFormat()
{
return new JsonInputFormat(null, null);
}
}

View File

@ -0,0 +1,487 @@
/*
* 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.base.Joiner;
import com.google.common.collect.Iterables;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
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.TaskToolbox;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.testing.junit.LoggerCaptureRule;
import org.apache.logging.log4j.core.LogEvent;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.hamcrest.Matchers;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.runners.Enclosed;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@RunWith(Enclosed.class)
public class PartialDimensionDistributionTaskTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
private static final SingleDimensionPartitionsSpec SINGLE_DIM_PARTITIONS_SPEC =
new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().build();
public static class ConstructorTest
{
@Rule
public ExpectedException exception = ExpectedException.none();
@Test
public void requiresForceGuaranteedRollup()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("forceGuaranteedRollup must be set");
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.forceGuaranteedRollup(false)
.partitionsSpec(new DynamicPartitionsSpec(null, null))
.build();
new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.build();
}
@Test
public void requiresSingleDimensionPartitions()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("single_dim partitionsSpec required");
PartitionsSpec partitionsSpec = new HashedPartitionsSpec(null, 1, null);
ParallelIndexTuningConfig tuningConfig =
new ParallelIndexTestingFactory.TuningConfigBuilder().partitionsSpec(partitionsSpec).build();
new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.build();
}
@Test
public void requiresGranularitySpecInputIntervals()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("Missing intervals in granularitySpec");
DataSchema dataSchema = ParallelIndexTestingFactory.createDataSchema(Collections.emptyList());
new PartialDimensionDistributionTaskBuilder()
.dataSchema(dataSchema)
.build();
}
@Test
public void serializesDeserializes()
{
PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder()
.build();
TestHelper.testSerializesDeserializes(OBJECT_MAPPER, task);
}
@Test
public void hasCorrectPrefixForAutomaticId()
{
PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder()
.id(ParallelIndexTestingFactory.AUTOMATIC_ID)
.build();
Assert.assertThat(task.getId(), Matchers.startsWith(PartialDimensionDistributionTask.TYPE));
}
}
public static class RunTaskTest
{
@Rule
public ExpectedException exception = ExpectedException.none();
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
@Rule
public LoggerCaptureRule logger = new LoggerCaptureRule(PartialDimensionDistributionTask.class);
private TaskToolbox taskToolbox;
@Before
public void setup()
{
taskToolbox = EasyMock.mock(TaskToolbox.class);
EasyMock.expect(taskToolbox.getIndexingTmpDir()).andStubReturn(temporaryFolder.getRoot());
EasyMock.replay(taskToolbox);
}
@Test
public void requiresPartitionDimension() throws Exception
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("partitionDimension must be specified");
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(
new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().partitionDimension(null).build()
)
.build();
PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.build();
task.runTask(taskToolbox);
}
@Test
public void logsParseExceptionsIfEnabled() throws Exception
{
long invalidTimestamp = Long.MAX_VALUE;
InputSource inlineInputSource = new InlineInputSource(
ParallelIndexTestingFactory.createRow(invalidTimestamp, "a")
);
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(SINGLE_DIM_PARTITIONS_SPEC)
.logParseExceptions(true)
.build();
PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder()
.inputSource(inlineInputSource)
.tuningConfig(tuningConfig)
.taskClientFactory(ParallelIndexTestingFactory.createTaskClientFactory())
.build();
task.runTask(taskToolbox);
List<LogEvent> logEvents = logger.getLogEvents();
Assert.assertEquals(1, logEvents.size());
String logMessage = logEvents.get(0).getMessage().getFormattedMessage();
Assert.assertThat(logMessage, Matchers.containsString("Encountered parse exception"));
}
@Test
public void doesNotLogParseExceptionsIfDisabled() throws Exception
{
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(SINGLE_DIM_PARTITIONS_SPEC)
.logParseExceptions(false)
.build();
PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.taskClientFactory(ParallelIndexTestingFactory.createTaskClientFactory())
.build();
task.runTask(taskToolbox);
Assert.assertEquals(Collections.emptyList(), logger.getLogEvents());
}
@Test
public void failsWhenTooManyParseExceptions() throws Exception
{
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(SINGLE_DIM_PARTITIONS_SPEC)
.maxParseExceptions(0)
.build();
PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.taskClientFactory(ParallelIndexTestingFactory.createTaskClientFactory())
.build();
exception.expect(RuntimeException.class);
exception.expectMessage("Max parse exceptions exceeded");
task.runTask(taskToolbox);
}
@Test
public void skipsRowsWithMultipleDimensionValues()
{
InputSource inlineInputSource = new InlineInputSource(
ParallelIndexTestingFactory.createRow(0, Arrays.asList("a", "b"))
);
PartialDimensionDistributionTaskBuilder taskBuilder = new PartialDimensionDistributionTaskBuilder()
.inputSource(inlineInputSource);
DimensionDistributionReport report = runTask(taskBuilder);
Map<Interval, StringDistribution> intervalToDistribution = report.getIntervalToDistribution();
Assert.assertEquals(0, intervalToDistribution.size());
}
@Test
public void sendsCorrectReportWhenAssumeGroupedTrue()
{
long timestamp = 0;
String dimensionValue = "a";
InputSource inlineInputSource = new InlineInputSource(
ParallelIndexTestingFactory.createRow(timestamp, dimensionValue)
+ "\n" + ParallelIndexTestingFactory.createRow(timestamp + 1, dimensionValue)
);
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(
new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().assumeGrouped(true).build()
)
.build();
PartialDimensionDistributionTaskBuilder taskBuilder = new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.inputSource(inlineInputSource);
DimensionDistributionReport report = runTask(taskBuilder);
Assert.assertEquals(ParallelIndexTestingFactory.ID, report.getTaskId());
Map<Interval, StringDistribution> intervalToDistribution = report.getIntervalToDistribution();
StringDistribution distribution = Iterables.getOnlyElement(intervalToDistribution.values());
Assert.assertNotNull(distribution);
PartitionBoundaries partitions = distribution.getEvenPartitionsByMaxSize(1);
Assert.assertEquals(2, partitions.size());
Assert.assertNull(partitions.get(0));
Assert.assertNull(partitions.get(1));
}
@Test
public void groupsRowsWhenAssumeGroupedFalse()
{
long timestamp = 0;
String dimensionValue = "a";
InputSource inlineInputSource = new InlineInputSource(
ParallelIndexTestingFactory.createRow(timestamp, dimensionValue)
+ "\n" + ParallelIndexTestingFactory.createRow(timestamp + 1, dimensionValue)
);
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(
new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().assumeGrouped(false).build()
)
.build();
PartialDimensionDistributionTaskBuilder taskBuilder = new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.inputSource(inlineInputSource);
DimensionDistributionReport report = runTask(taskBuilder);
Assert.assertEquals(ParallelIndexTestingFactory.ID, report.getTaskId());
Map<Interval, StringDistribution> intervalToDistribution = report.getIntervalToDistribution();
StringDistribution distribution = Iterables.getOnlyElement(intervalToDistribution.values());
Assert.assertNotNull(distribution);
PartitionBoundaries partitions = distribution.getEvenPartitionsByMaxSize(1);
Assert.assertEquals(2, partitions.size());
Assert.assertNull(partitions.get(0));
Assert.assertNull(partitions.get(1));
}
@Test
public void preservesMinAndMaxWhenAssumeGroupedFalse()
{
// Create a small bloom filter so that it saturates quickly
int smallBloomFilter = 1;
double manyFalsePositiveBloomFilter = 0.5;
int minBloomFilterBits = Long.SIZE;
long timestamp = 0;
List<String> dimensionValues = IntStream.range(0, minBloomFilterBits * 10)
.mapToObj(i -> StringUtils.format("%010d", i))
.collect(Collectors.toCollection(ArrayList::new));
List<String> rows = dimensionValues.stream()
.map(d -> ParallelIndexTestingFactory.createRow(timestamp, d))
.collect(Collectors.toList());
Joiner joiner = Joiner.on("\n");
InputSource inlineInputSource = new InlineInputSource(
joiner.join(
joiner.join(rows.subList(1, rows.size())), // saturate bloom filter first
rows.get(0),
rows.get(rows.size() - 1)
)
);
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(
new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().assumeGrouped(false).build()
)
.build();
DataSchema dataSchema = ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS);
PartialDimensionDistributionTaskBuilder taskBuilder = new PartialDimensionDistributionTaskBuilder()
.tuningConfig(tuningConfig)
.dataSchema(dataSchema)
.inputSource(inlineInputSource)
.dedupRowDimValueFilterSupplier(
() -> new PartialDimensionDistributionTask.DedupRowDimensionValueFilter(
dataSchema.getGranularitySpec().getQueryGranularity(),
smallBloomFilter,
manyFalsePositiveBloomFilter
)
);
DimensionDistributionReport report = runTask(taskBuilder);
Assert.assertEquals(ParallelIndexTestingFactory.ID, report.getTaskId());
Map<Interval, StringDistribution> intervalToDistribution = report.getIntervalToDistribution();
StringDistribution distribution = Iterables.getOnlyElement(intervalToDistribution.values());
Assert.assertNotNull(distribution);
PartitionBoundaries partitions = distribution.getEvenPartitionsByMaxSize(1);
Assert.assertEquals(minBloomFilterBits + 2, partitions.size()); // 2 = min + max
String minDimensionValue = dimensionValues.get(0);
Assert.assertEquals(minDimensionValue, ((StringSketch) distribution).getMin());
String maxDimensionValue = dimensionValues.get(dimensionValues.size() - 1);
Assert.assertEquals(maxDimensionValue, ((StringSketch) distribution).getMax());
}
@Test
public void returnsSuccessIfNoExceptions() throws Exception
{
PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder()
.taskClientFactory(ParallelIndexTestingFactory.createTaskClientFactory())
.build();
TaskStatus taskStatus = task.runTask(taskToolbox);
Assert.assertEquals(ParallelIndexTestingFactory.ID, taskStatus.getId());
Assert.assertEquals(TaskState.SUCCESS, taskStatus.getStatusCode());
}
private DimensionDistributionReport runTask(PartialDimensionDistributionTaskBuilder taskBuilder)
{
Capture<SubTaskReport> reportCapture = Capture.newInstance();
ParallelIndexSupervisorTaskClient taskClient = EasyMock.mock(ParallelIndexSupervisorTaskClient.class);
taskClient.report(EasyMock.eq(ParallelIndexTestingFactory.SUPERVISOR_TASK_ID), EasyMock.capture(reportCapture));
EasyMock.replay(taskClient);
try {
taskBuilder.taskClientFactory((taskInfoProvider, callerId, numThreads, httpTimeout, numRetries) -> taskClient)
.build()
.runTask(taskToolbox);
}
catch (Exception e) {
throw new RuntimeException(e);
}
return (DimensionDistributionReport) reportCapture.getValue();
}
}
private static class PartialDimensionDistributionTaskBuilder
{
private static final InputFormat INPUT_FORMAT = ParallelIndexTestingFactory.getInputFormat();
private String id = ParallelIndexTestingFactory.ID;
private InputSource inputSource = new InlineInputSource("row-with-invalid-timestamp");
private ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().build())
.build();
private DataSchema dataSchema =
ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS);
private IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory =
ParallelIndexTestingFactory.TASK_CLIENT_FACTORY;
private Supplier<PartialDimensionDistributionTask.DedupRowDimensionValueFilter> dedupRowDimValueFilterSupplier =
null;
@SuppressWarnings("SameParameterValue")
PartialDimensionDistributionTaskBuilder id(String id)
{
this.id = id;
return this;
}
PartialDimensionDistributionTaskBuilder inputSource(InputSource inputSource)
{
this.inputSource = inputSource;
return this;
}
PartialDimensionDistributionTaskBuilder tuningConfig(ParallelIndexTuningConfig tuningConfig)
{
this.tuningConfig = tuningConfig;
return this;
}
PartialDimensionDistributionTaskBuilder dataSchema(DataSchema dataSchema)
{
this.dataSchema = dataSchema;
return this;
}
PartialDimensionDistributionTaskBuilder taskClientFactory(
IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory
)
{
this.taskClientFactory = taskClientFactory;
return this;
}
PartialDimensionDistributionTaskBuilder dedupRowDimValueFilterSupplier(
Supplier<PartialDimensionDistributionTask.DedupRowDimensionValueFilter> dedupRowDimValueFilterSupplier
)
{
this.dedupRowDimValueFilterSupplier = dedupRowDimValueFilterSupplier;
return this;
}
PartialDimensionDistributionTask build()
{
ParallelIndexIngestionSpec ingestionSpec =
ParallelIndexTestingFactory.createIngestionSpec(inputSource, INPUT_FORMAT, tuningConfig, dataSchema);
Supplier<PartialDimensionDistributionTask.DedupRowDimensionValueFilter> supplier =
dedupRowDimValueFilterSupplier == null
? () -> new PartialDimensionDistributionTask.DedupRowDimensionValueFilter(
dataSchema.getGranularitySpec().getQueryGranularity()
)
: dedupRowDimValueFilterSupplier;
return new PartialDimensionDistributionTask(
id,
ParallelIndexTestingFactory.GROUP_ID,
ParallelIndexTestingFactory.TASK_RESOURCE,
ParallelIndexTestingFactory.SUPERVISOR_TASK_ID,
ParallelIndexTestingFactory.NUM_ATTEMPTS,
ingestionSpec,
ParallelIndexTestingFactory.CONTEXT,
ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT,
taskClientFactory,
supplier
);
}
}
}

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 PartialGenericSegmentMergeIOConfigTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
private static final GenericPartitionLocation GENERIC_PARTITION_LOCATION = new GenericPartitionLocation(
ParallelIndexTestingFactory.HOST,
ParallelIndexTestingFactory.PORT,
ParallelIndexTestingFactory.USE_HTTPS,
ParallelIndexTestingFactory.SUBTASK_ID,
ParallelIndexTestingFactory.INTERVAL,
ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC
);
private PartialGenericSegmentMergeIOConfig target;
@Before
public void setup()
{
target = new PartialGenericSegmentMergeIOConfig(Collections.singletonList(GENERIC_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 PartialGenericSegmentMergeIngestionSpecTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
private static final GenericPartitionLocation GENERIC_PARTITION_LOCATION = new GenericPartitionLocation(
ParallelIndexTestingFactory.HOST,
ParallelIndexTestingFactory.PORT,
ParallelIndexTestingFactory.USE_HTTPS,
ParallelIndexTestingFactory.SUBTASK_ID,
ParallelIndexTestingFactory.INTERVAL,
ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC
);
private static final PartialGenericSegmentMergeIOConfig IO_CONFIG =
new PartialGenericSegmentMergeIOConfig(Collections.singletonList(GENERIC_PARTITION_LOCATION));
private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec(
null,
1,
Collections.emptyList()
);
private PartialGenericSegmentMergeIngestionSpec target;
@Before
public void setup()
{
target = new PartialGenericSegmentMergeIngestionSpec(
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 PartialGenericSegmentMergeTaskTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
private static final GenericPartitionLocation GENERIC_PARTITION_LOCATION = new GenericPartitionLocation(
ParallelIndexTestingFactory.HOST,
ParallelIndexTestingFactory.PORT,
ParallelIndexTestingFactory.USE_HTTPS,
ParallelIndexTestingFactory.SUBTASK_ID,
ParallelIndexTestingFactory.INTERVAL,
ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC
);
private static final PartialGenericSegmentMergeIOConfig IO_CONFIG =
new PartialGenericSegmentMergeIOConfig(Collections.singletonList(GENERIC_PARTITION_LOCATION));
private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec(
null,
1,
Collections.emptyList()
);
private static final PartialGenericSegmentMergeIngestionSpec INGESTION_SPEC =
new PartialGenericSegmentMergeIngestionSpec(
ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS),
IO_CONFIG,
new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(PARTITIONS_SPEC)
.build()
);
private PartialGenericSegmentMergeTask target;
@Before
public void setup()
{
target = new PartialGenericSegmentMergeTask(
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(PartialGenericSegmentMergeTask.TYPE));
}
}

View File

@ -0,0 +1,154 @@
/*
* 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.ImmutableMap;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.indexing.DataSchema;
import org.hamcrest.Matchers;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.util.Collections;
public class PartialRangeSegmentGenerateTaskTest
{
private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper();
@Rule
public ExpectedException exception = ExpectedException.none();
@Test
public void requiresForceGuaranteedRollup()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("single_dim partitionsSpec required");
ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.forceGuaranteedRollup(false)
.partitionsSpec(new DynamicPartitionsSpec(null, null))
.build();
new PartialRangeSegmentGenerateTaskBuilder()
.tuningConfig(tuningConfig)
.build();
}
@Test
public void requiresSingleDimensionPartitions()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("single_dim partitionsSpec required");
PartitionsSpec partitionsSpec = new HashedPartitionsSpec(null, 1, null);
ParallelIndexTuningConfig tuningConfig =
new ParallelIndexTestingFactory.TuningConfigBuilder().partitionsSpec(partitionsSpec).build();
new PartialRangeSegmentGenerateTaskBuilder()
.tuningConfig(tuningConfig)
.build();
}
@Test
public void requiresGranularitySpecInputIntervals()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("Missing intervals in granularitySpec");
DataSchema dataSchema = ParallelIndexTestingFactory.createDataSchema(Collections.emptyList());
new PartialRangeSegmentGenerateTaskBuilder()
.dataSchema(dataSchema)
.build();
}
@Test
public void serializesDeserializes()
{
PartialRangeSegmentGenerateTask task = new PartialRangeSegmentGenerateTaskBuilder().build();
TestHelper.testSerializesDeserializes(OBJECT_MAPPER, task);
}
@Test
public void hasCorrectPrefixForAutomaticId()
{
PartialRangeSegmentGenerateTask task = new PartialRangeSegmentGenerateTaskBuilder().build();
Assert.assertThat(task.getId(), Matchers.startsWith(PartialRangeSegmentGenerateTask.TYPE));
}
private static class PartialRangeSegmentGenerateTaskBuilder
{
private static final InputSource INPUT_SOURCE = new InlineInputSource("data");
private static final InputFormat INPUT_FORMAT = ParallelIndexTestingFactory.getInputFormat();
private final IndexTaskClientFactory<ParallelIndexSupervisorTaskClient> taskClientFactory =
ParallelIndexTestingFactory.TASK_CLIENT_FACTORY;
private ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder()
.partitionsSpec(new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().build())
.build();
private DataSchema dataSchema =
ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS);
PartialRangeSegmentGenerateTaskBuilder tuningConfig(ParallelIndexTuningConfig tuningConfig)
{
this.tuningConfig = tuningConfig;
return this;
}
PartialRangeSegmentGenerateTaskBuilder dataSchema(DataSchema dataSchema)
{
this.dataSchema = dataSchema;
return this;
}
PartialRangeSegmentGenerateTask build()
{
ParallelIndexIngestionSpec ingestionSpec =
ParallelIndexTestingFactory.createIngestionSpec(INPUT_SOURCE, INPUT_FORMAT, tuningConfig, dataSchema);
return new PartialRangeSegmentGenerateTask(
ParallelIndexTestingFactory.AUTOMATIC_ID,
ParallelIndexTestingFactory.GROUP_ID,
ParallelIndexTestingFactory.TASK_RESOURCE,
ParallelIndexTestingFactory.SUPERVISOR_TASK_ID,
ParallelIndexTestingFactory.NUM_ATTEMPTS,
ingestionSpec,
ParallelIndexTestingFactory.CONTEXT,
ImmutableMap.of(Intervals.ETERNITY, new PartitionBoundaries("a")),
ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT,
taskClientFactory,
ParallelIndexTestingFactory.APPENDERATORS_MANAGER
);
}
}
}

View File

@ -0,0 +1,475 @@
/*
* 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.ArrayListMultimap;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Multimap;
import com.google.common.collect.SetMultimap;
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.SingleDimensionPartitionsSpec;
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.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
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.guava.Comparators;
import org.apache.druid.query.scan.ScanResultValue;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.hamcrest.Matchers;
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.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.stream.Collectors;
@RunWith(Parameterized.class)
public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiPhaseParallelIndexingTest
{
private static final int NUM_FILE = 10;
private static final int NUM_ROW = 20;
private static final int NUM_DAY = 2;
private static final int NUM_PARTITION = 2;
private static final int YEAR = 2017;
private static final String DIM1 = "dim1";
private static final String DIM2 = "dim2";
private static final List<String> DIMS = ImmutableList.of(DIM1, DIM2);
private static final String TEST_FILE_NAME_PREFIX = "test_";
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;
private SetMultimap<Interval, String> intervalToDim1;
public RangePartitionMultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi)
{
super(lockGranularity, useInputFormatApi);
}
@Override
@Before
public void setup() throws IOException
{
super.setup();
inputDir = temporaryFolder.newFolder("data");
intervalToDim1 = createInputFiles(inputDir);
}
private static SetMultimap<Interval, String> createInputFiles(File inputDir) throws IOException
{
SetMultimap<Interval, String> intervalToDim1 = HashMultimap.create();
for (int fileIndex = 0; fileIndex < NUM_FILE; fileIndex++) {
Path path = new File(inputDir, TEST_FILE_NAME_PREFIX + fileIndex).toPath();
try (final Writer writer = Files.newBufferedWriter(path, StandardCharsets.UTF_8)) {
for (int i = 0; i < (NUM_ROW / NUM_DAY); i++) {
for (int d = 0; d < NUM_DAY; d++) {
writeRow(writer, i + d, fileIndex + d, intervalToDim1);
}
}
}
}
return intervalToDim1;
}
private static void writeRow(Writer writer, int day, int fileIndex, Multimap<Interval, String> intervalToDim1)
throws IOException
{
Interval interval = Intervals.of("%s-12-%d/%s-12-%d", YEAR, day + 1, YEAR, day + 2);
String startDate = interval.getStart().toString("y-M-d");
String dim1Value = String.valueOf(fileIndex + 10);
writer.write(StringUtils.format("%s,%s,%d th test file\n", startDate, dim1Value, fileIndex));
intervalToDim1.put(interval, dim1Value);
}
@Test
public void createsCorrectRangePartitions() throws Exception
{
int targetRowsPerSegment = NUM_ROW / NUM_DAY / NUM_PARTITION;
final Set<DataSegment> publishedSegments = runTestTask(
PARSE_SPEC,
Intervals.of("%s/%s", YEAR, YEAR + 1),
inputDir,
TEST_FILE_NAME_PREFIX + "*",
new SingleDimensionPartitionsSpec(
targetRowsPerSegment,
null,
DIM1,
false
)
);
assertRangePartitions(publishedSegments);
}
private void assertRangePartitions(Set<DataSegment> publishedSegments) throws IOException
{
Multimap<Interval, DataSegment> intervalToSegments = ArrayListMultimap.create();
publishedSegments.forEach(s -> intervalToSegments.put(s.getInterval(), s));
SortedSet<Interval> publishedIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
publishedIntervals.addAll(intervalToSegments.keySet());
assertHasExpectedIntervals(publishedIntervals);
Interval firstInterval = publishedIntervals.first();
Interval lastInterval = publishedIntervals.last();
File tempSegmentDir = temporaryFolder.newFolder();
intervalToSegments.asMap().forEach((interval, segments) -> {
assertNumPartition(interval, segments, firstInterval, lastInterval);
List<String> allValues = new ArrayList<>(NUM_ROW);
for (DataSegment segment : segments) {
List<String> values = getColumnValues(segment, tempSegmentDir);
assertValuesInRange(values, segment);
allValues.addAll(values);
}
assertIntervalHasAllExpectedValues(interval, allValues);
});
}
private void assertHasExpectedIntervals(Set<Interval> publishedSegmentIntervals)
{
Assert.assertEquals(intervalToDim1.keySet(), publishedSegmentIntervals);
}
private static void assertNumPartition(
Interval interval,
Collection<DataSegment> segments,
Interval firstInterval,
Interval lastInterval
)
{
int expectedNumPartition = NUM_PARTITION;
if (interval.equals(firstInterval) || interval.equals(lastInterval)) {
expectedNumPartition -= 1;
}
expectedNumPartition *= NUM_DAY;
Assert.assertEquals(expectedNumPartition, segments.size());
}
private List<String> getColumnValues(DataSegment segment, File tempDir)
{
List<ScanResultValue> results = querySegment(segment, DIMS, tempDir);
Assert.assertEquals(1, results.size());
List<LinkedHashMap<String, String>> rows = (List<LinkedHashMap<String, String>>) results.get(0).getEvents();
return rows.stream()
.map(row -> row.get(DIM1))
.collect(Collectors.toList());
}
private static void assertValuesInRange(List<String> values, DataSegment segment)
{
SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segment.getShardSpec();
String start = shardSpec.getStart();
String end = shardSpec.getEnd();
Assert.assertTrue(shardSpec.toString(), start != null || end != null);
for (String value : values) {
if (start != null) {
Assert.assertThat(value.compareTo(start), Matchers.greaterThanOrEqualTo(0));
}
if (end != null) {
Assert.assertThat(value.compareTo(end), Matchers.lessThan(0));
}
}
}
private void assertIntervalHasAllExpectedValues(Interval interval, List<String> actualValues)
{
List<String> expectedValues = new ArrayList<>(intervalToDim1.get(interval));
Assert.assertEquals(expectedValues.size(), actualValues.size());
Collections.sort(expectedValues);
Collections.sort(actualValues);
Assert.assertEquals(expectedValues, actualValues);
}
@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
PartialDimensionDistributionParallelIndexTaskRunner createPartialDimensionDistributionRunner(TaskToolbox toolbox)
{
return new TestPartialDimensionDistributionRunner(toolbox, this, getIndexingServiceClient());
}
@Override
PartialRangeSegmentGenerateParallelIndexTaskRunner createPartialRangeSegmentGenerateRunner(
TaskToolbox toolbox,
Map<Interval, PartitionBoundaries> intervalToPartitions
)
{
return new TestPartialRangeSegmentGenerateRunner(
toolbox,
this,
getIndexingServiceClient(),
intervalToPartitions
);
}
@Override
public PartialGenericSegmentMergeParallelIndexTaskRunner createPartialGenericSegmentMergeRunner(
TaskToolbox toolbox,
List<PartialGenericSegmentMergeIOConfig> ioConfigs
)
{
return new TestPartialGenericSegmentMergeParallelIndexTaskRunner(
toolbox,
this,
ioConfigs,
getIndexingServiceClient()
);
}
}
private static class TestPartialDimensionDistributionRunner
extends PartialDimensionDistributionParallelIndexTaskRunner
{
private TestPartialDimensionDistributionRunner(
TaskToolbox toolbox,
ParallelIndexSupervisorTask supervisorTask,
IndexingServiceClient indexingServiceClient
)
{
super(
toolbox,
supervisorTask.getId(),
supervisorTask.getGroupId(),
supervisorTask.getIngestionSchema(),
supervisorTask.getContext(),
indexingServiceClient,
new LocalParallelIndexTaskClientFactory(supervisorTask)
);
}
}
private static class TestPartialRangeSegmentGenerateRunner extends PartialRangeSegmentGenerateParallelIndexTaskRunner
{
private TestPartialRangeSegmentGenerateRunner(
TaskToolbox toolbox,
ParallelIndexSupervisorTask supervisorTask,
IndexingServiceClient indexingServiceClient,
Map<Interval, PartitionBoundaries> intervalToPartitions
)
{
super(
toolbox,
supervisorTask.getId(),
supervisorTask.getGroupId(),
supervisorTask.getIngestionSchema(),
supervisorTask.getContext(),
indexingServiceClient,
intervalToPartitions,
new LocalParallelIndexTaskClientFactory(supervisorTask),
new TestAppenderatorsManager()
);
}
}
private static class TestPartialGenericSegmentMergeParallelIndexTaskRunner
extends PartialGenericSegmentMergeParallelIndexTaskRunner
{
private final ParallelIndexSupervisorTask supervisorTask;
private TestPartialGenericSegmentMergeParallelIndexTaskRunner(
TaskToolbox toolbox,
ParallelIndexSupervisorTask supervisorTask,
List<PartialGenericSegmentMergeIOConfig> mergeIOConfigs,
IndexingServiceClient indexingServiceClient
)
{
super(
toolbox,
supervisorTask.getId(),
supervisorTask.getGroupId(),
supervisorTask.getIngestionSchema().getDataSchema(),
mergeIOConfigs,
supervisorTask.getIngestionSchema().getTuningConfig(),
supervisorTask.getContext(),
indexingServiceClient
);
this.supervisorTask = supervisorTask;
}
@Override
SubTaskSpec<PartialGenericSegmentMergeTask> newTaskSpec(PartialGenericSegmentMergeIOConfig ioConfig)
{
final PartialGenericSegmentMergeIngestionSpec ingestionSpec =
new PartialGenericSegmentMergeIngestionSpec(
supervisorTask.getIngestionSchema().getDataSchema(),
ioConfig,
getTuningConfig()
);
return new SubTaskSpec<PartialGenericSegmentMergeTask>(
getTaskId() + "_" + getAndIncrementNextSpecId(),
getGroupId(),
getTaskId(),
getContext(),
new InputSplit<>(ioConfig.getPartitionLocations())
)
{
@Override
public PartialGenericSegmentMergeTask newSubTask(int numAttempts)
{
return new TestPartialGenericSegmentMergeTask(
null,
getGroupId(),
null,
getSupervisorTaskId(),
numAttempts,
ingestionSpec,
getContext(),
getIndexingServiceClient(),
new LocalParallelIndexTaskClientFactory(supervisorTask),
getToolbox()
);
}
};
}
}
private static class TestPartialGenericSegmentMergeTask extends PartialGenericSegmentMergeTask
{
private final TaskToolbox toolbox;
private TestPartialGenericSegmentMergeTask(
@Nullable String id,
String groupId,
TaskResource taskResource,
String supervisorTaskId,
int numAttempts,
PartialGenericSegmentMergeIngestionSpec 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, GenericPartitionLocation 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,81 @@
/*
* 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.distribution;
import org.apache.druid.segment.TestHelper;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
public class PartitionBoundariesTest
{
private PartitionBoundaries target;
private String[] values;
private List<String> expected;
@Before
public void setup()
{
values = new String[]{"a", "dup", "dup", "z"};
expected = Arrays.asList(null, "dup", null);
target = new PartitionBoundaries(values);
}
@Test
public void hasCorrectValues()
{
Assert.assertEquals(expected, target);
}
@Test(expected = UnsupportedOperationException.class)
public void isImmutable()
{
target.add("should fail");
}
@Test
public void cannotBeIndirectlyModified()
{
values[1] = "changed";
Assert.assertEquals(expected, target);
}
@Test
public void handlesNoValues()
{
Assert.assertEquals(Collections.emptyList(), new PartitionBoundaries());
}
@Test
public void handlesRepeatedValue()
{
Assert.assertEquals(Arrays.asList(null, null), new PartitionBoundaries("a", "a", "a"));
}
@Test
public void serializesDeserializes()
{
TestHelper.testSerializesDeserializes(TestHelper.JSON_MAPPER, target);
}
}

View File

@ -0,0 +1,79 @@
/*
* 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.distribution;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
public class StringSketchMergerTest
{
private StringSketchMerger target;
@Rule
public ExpectedException exception = ExpectedException.none();
@Before
public void setup()
{
target = new StringSketchMerger();
}
@Test
public void requiresStringSketch()
{
StringDistribution distribution = EasyMock.mock(StringDistribution.class);
exception.expect(IllegalArgumentException.class);
exception.expectMessage("Only merging StringSketch instances is currently supported");
target.merge(distribution);
}
@Test
public void mergesCorrectly()
{
String string1 = "a";
StringSketch sketch1 = new StringSketch();
sketch1.put(string1);
String string2 = "mn";
StringSketch sketch2 = new StringSketch();
sketch2.put(string2);
String string3 = "z";
StringSketch sketch3 = new StringSketch();
sketch3.put(string3);
target.merge(sketch2);
target.merge(sketch1);
target.merge(sketch3);
StringDistribution merged = target.getResult();
PartitionBoundaries partitions = merged.getEvenPartitionsByMaxSize(1);
Assert.assertEquals(3, partitions.size());
Assert.assertNull(partitions.get(0));
Assert.assertEquals(string2, partitions.get(1));
Assert.assertNull(partitions.get(2));
}
}

View File

@ -0,0 +1,380 @@
/*
* 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.distribution;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.datasketches.quantiles.ItemsSketch;
import org.apache.druid.jackson.JacksonModule;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.TestHelper;
import org.hamcrest.Matchers;
import org.hamcrest.number.IsCloseTo;
import org.junit.Assert;
import org.junit.Before;
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.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.StringJoiner;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@RunWith(Enclosed.class)
public class StringSketchTest
{
private static final int FACTOR = 2;
private static final int NUM_STRING = StringSketch.SKETCH_K * FACTOR;
private static final double DELTA = ItemsSketch.getNormalizedRankError(StringSketch.SKETCH_K, true) * NUM_STRING;
private static final List<String> STRINGS = IntStream.range(0, NUM_STRING)
.mapToObj(i -> StringUtils.format("%010d", i))
.collect(Collectors.toCollection(ArrayList::new));
private static final String MIN_STRING = STRINGS.get(0);
private static final String MAX_STRING = STRINGS.get(NUM_STRING - 1);
static {
ItemsSketch.rand.setSeed(0); // make sketches deterministic for testing
}
public static class SerializationDeserializationTest
{
private static final ObjectMapper OBJECT_MAPPER = new JacksonModule().smileMapper();
@Test
public void serializesDeserializes()
{
StringSketch target = new StringSketch();
target.put(MIN_STRING);
target.put(MAX_STRING);
TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target);
}
}
public static class PutTest
{
private StringSketch target;
@Before
public void setup()
{
target = new StringSketch();
}
@Test
public void putIfNewMin()
{
String value = MAX_STRING;
Assert.assertEquals(0, getCount());
target.putIfNewMin(value);
Assert.assertEquals(1, getCount());
target.putIfNewMin(value);
Assert.assertEquals(1, getCount());
Assert.assertEquals(value, target.getDelegate().getMinValue());
Assert.assertEquals(value, target.getDelegate().getMaxValue());
target.putIfNewMin(MIN_STRING);
Assert.assertEquals(2, getCount());
Assert.assertEquals(MIN_STRING, target.getDelegate().getMinValue());
Assert.assertEquals(MAX_STRING, target.getDelegate().getMaxValue());
}
@Test
public void putIfNewMax()
{
String value = MIN_STRING;
Assert.assertEquals(0, getCount());
target.putIfNewMax(value);
Assert.assertEquals(1, getCount());
target.putIfNewMax(value);
Assert.assertEquals(1, getCount());
Assert.assertEquals(value, target.getDelegate().getMinValue());
Assert.assertEquals(value, target.getDelegate().getMaxValue());
target.putIfNewMax(MAX_STRING);
Assert.assertEquals(2, getCount());
Assert.assertEquals(MIN_STRING, target.getDelegate().getMinValue());
Assert.assertEquals(MAX_STRING, target.getDelegate().getMaxValue());
}
private long getCount()
{
return target.getDelegate().getN();
}
}
@RunWith(Enclosed.class)
public static class PartitionTest
{
private static final StringSketch SKETCH;
static {
SKETCH = new StringSketch();
STRINGS.forEach(SKETCH::put);
}
public static class TargetSizeTest
{
@Rule
public ExpectedException exception = ExpectedException.none();
@Test
public void requiresPositiveSize()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("targetSize must be positive but is 0");
SKETCH.getEvenPartitionsByTargetSize(0);
}
@Test
public void handlesEmptySketch()
{
StringSketch sketch = new StringSketch();
PartitionBoundaries partitionBoundaries = sketch.getEvenPartitionsByTargetSize(1);
Assert.assertEquals(0, partitionBoundaries.size());
}
@Test
public void handlesSingletonSketch()
{
StringSketch sketch = new StringSketch();
sketch.put(MIN_STRING);
PartitionBoundaries partitionBoundaries = sketch.getEvenPartitionsByTargetSize(1);
Assert.assertEquals(2, partitionBoundaries.size());
Assert.assertNull(partitionBoundaries.get(0));
Assert.assertNull(partitionBoundaries.get(1));
}
@Test
public void handlesMinimimumSize()
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByTargetSize(1);
assertMaxNumberOfPartitions(partitionBoundaries);
}
@Test
public void handlesUnevenPartitions()
{
List<Integer> targetSizes = Arrays.asList(127, 257, 509, 1021, 2039, 4093);
targetSizes.forEach(TargetSizeTest::testHandlesUnevenPartitions);
}
private static void testHandlesUnevenPartitions(int targetSize)
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByTargetSize(targetSize);
assertFirstAndLastPartitionsCorrect(partitionBoundaries);
String partitionBoundariesString = PartitionTest.toString(partitionBoundaries);
int expectedHighPartitionBoundaryCount = (int) Math.ceil((double) NUM_STRING / targetSize);
int expectedLowPartitionBoundaryCount = expectedHighPartitionBoundaryCount - 1;
Assert.assertThat(
"targetSize=" + targetSize + " " + partitionBoundariesString,
partitionBoundaries.size(),
Matchers.lessThanOrEqualTo(expectedHighPartitionBoundaryCount + 1)
);
Assert.assertThat(
"targetSize=" + targetSize + " " + partitionBoundariesString,
partitionBoundaries.size(),
Matchers.greaterThanOrEqualTo(expectedLowPartitionBoundaryCount + 1)
);
int previous = 0;
for (int i = 1; i < partitionBoundaries.size() - 1; i++) {
int current = Integer.parseInt(partitionBoundaries.get(i));
int size = current - previous;
Assert.assertThat(
getErrMsgPrefix(targetSize, i) + partitionBoundariesString,
(double) size,
IsCloseTo.closeTo(targetSize, Math.ceil(DELTA) * 2)
);
previous = current;
}
}
@Test
public void handlesSinglePartition()
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByTargetSize(NUM_STRING);
assertSinglePartition(partitionBoundaries);
}
@Test
public void handlesOversizedPartition()
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByTargetSize(Integer.MAX_VALUE);
assertSinglePartition(partitionBoundaries);
}
}
public static class MaxSizeTest
{
@Rule
public ExpectedException exception = ExpectedException.none();
@Test
public void requiresPositiveSize()
{
exception.expect(IllegalArgumentException.class);
exception.expectMessage("maxSize must be positive but is 0");
SKETCH.getEvenPartitionsByMaxSize(0);
}
@Test
public void handlesEmptySketch()
{
StringSketch sketch = new StringSketch();
PartitionBoundaries partitionBoundaries = sketch.getEvenPartitionsByMaxSize(1);
Assert.assertEquals(0, partitionBoundaries.size());
}
@Test
public void handlesSingletonSketch()
{
StringSketch sketch = new StringSketch();
sketch.put(MIN_STRING);
PartitionBoundaries partitionBoundaries = sketch.getEvenPartitionsByMaxSize(1);
Assert.assertEquals(2, partitionBoundaries.size());
Assert.assertNull(partitionBoundaries.get(0));
Assert.assertNull(partitionBoundaries.get(1));
}
@Test
public void handlesMinimimumSize()
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByMaxSize(1);
assertMaxNumberOfPartitions(partitionBoundaries);
}
@Test
public void handlesUnevenPartitions()
{
List<Integer> maxSizes = Arrays.asList(509, 1021, 2039, 4093);
maxSizes.forEach(MaxSizeTest::testHandlesUnevenPartitions);
}
private static void testHandlesUnevenPartitions(int maxSize)
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByMaxSize(maxSize);
assertFirstAndLastPartitionsCorrect(partitionBoundaries);
String partitionBoundariesString = PartitionTest.toString(partitionBoundaries);
long expectedPartitionCount = (long) Math.ceil((double) NUM_STRING / maxSize);
Assert.assertEquals(
"maxSize=" + maxSize + " " + partitionBoundariesString,
expectedPartitionCount + 1,
partitionBoundaries.size()
);
double minSize = (double) NUM_STRING / expectedPartitionCount - DELTA;
int previous = 0;
for (int i = 1; i < partitionBoundaries.size() - 1; i++) {
int current = Integer.parseInt(partitionBoundaries.get(i));
int size = current - previous;
Assert.assertThat(
getErrMsgPrefix(maxSize, i) + partitionBoundariesString,
size,
Matchers.lessThanOrEqualTo(maxSize)
);
Assert.assertThat(
getErrMsgPrefix(maxSize, i) + partitionBoundariesString,
(double) size,
Matchers.greaterThanOrEqualTo(minSize)
);
previous = current;
}
}
@Test
public void handlesSinglePartition()
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByMaxSize(
(int) Math.ceil(NUM_STRING + DELTA)
);
assertSinglePartition(partitionBoundaries);
}
@Test
public void handlesOversizedPartition()
{
PartitionBoundaries partitionBoundaries = SKETCH.getEvenPartitionsByMaxSize(Integer.MAX_VALUE);
assertSinglePartition(partitionBoundaries);
}
}
private static void assertMaxNumberOfPartitions(PartitionBoundaries partitionBoundaries)
{
String partitionBoundariesString = toString(partitionBoundaries);
Assert.assertEquals(partitionBoundariesString, StringSketch.SKETCH_K + 1, partitionBoundaries.size());
assertFirstAndLastPartitionsCorrect(partitionBoundaries);
int previous = 0;
for (int i = 1; i < partitionBoundaries.size() - 1; i++) {
int current = Integer.parseInt(partitionBoundaries.get(i));
Assert.assertEquals(
getErrMsgPrefix(1, i) + partitionBoundariesString,
1,
current - previous,
FACTOR
);
previous = current;
}
}
private static void assertSinglePartition(PartitionBoundaries partitionBoundaries)
{
Assert.assertEquals(2, partitionBoundaries.size());
assertFirstAndLastPartitionsCorrect(partitionBoundaries);
}
private static void assertFirstAndLastPartitionsCorrect(PartitionBoundaries partitionBoundaries)
{
Assert.assertNull(partitionBoundaries.get(0));
Assert.assertNull(partitionBoundaries.get(partitionBoundaries.size() - 1));
}
private static String getErrMsgPrefix(int size, int i)
{
return "size=" + size + " i=" + i + " of ";
}
private static String toString(PartitionBoundaries partitionBoundaries)
{
String prefix = "partitionBoundaries[" + partitionBoundaries.size() + "]=";
StringJoiner sj = new StringJoiner(" ", prefix, "]");
for (int i = 0; i < partitionBoundaries.size(); i++) {
sj.add("[" + i + "]=" + partitionBoundaries.get(i));
}
return sj.toString();
}
}
}

View File

@ -0,0 +1,66 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.indexing.common.task.batch.parallel.distribution;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TimeDimTupleFactoryTest
{
private static final Granularity GRANULARITY = Granularities.SECOND;
private static final DateTime TIMESTAMP = DateTimes.utc(0);
private static final String DIMENSION_VALUE = "abc";
private TimeDimTupleFactory target;
@Before
public void setup()
{
target = new TimeDimTupleFactory(GRANULARITY);
}
@Test
public void adjustsTimestamps()
{
TimeDimTuple timeDimTuple = target.createWithBucketedTimestamp(TIMESTAMP, DIMENSION_VALUE);
Assert.assertEquals(TIMESTAMP.getMillis(), timeDimTuple.getTimestamp());
TimeDimTuple timeDimTuple_plus_1msec = target.createWithBucketedTimestamp(TIMESTAMP.plus(1), DIMENSION_VALUE);
Assert.assertEquals(TIMESTAMP.getMillis(), timeDimTuple_plus_1msec.getTimestamp());
TimeDimTuple timeDimTuple_plus_999msec = target.createWithBucketedTimestamp(TIMESTAMP.plus(999), DIMENSION_VALUE);
Assert.assertEquals(TIMESTAMP.getMillis(), timeDimTuple_plus_999msec.getTimestamp());
TimeDimTuple timeDimTuple_plus_1sec = target.createWithBucketedTimestamp(TIMESTAMP.plus(1000), DIMENSION_VALUE);
Assert.assertEquals(TIMESTAMP.getMillis() + 1000, timeDimTuple_plus_1sec.getTimestamp());
}
@Test
public void setsDimensionValue()
{
TimeDimTuple timeDimTuple = target.createWithBucketedTimestamp(TIMESTAMP, DIMENSION_VALUE);
Assert.assertEquals(DIMENSION_VALUE, timeDimTuple.getDimensionValue());
}
}

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.distribution;
import com.google.common.hash.BloomFilter;
import org.junit.Assert;
import org.junit.Test;
public class TimeDimTupleFunnelTest
{
@Test
public void worksWithBloomFilter()
{
TimeDimTuple tuple = new TimeDimTuple(1000, "a");
BloomFilter<TimeDimTuple> bloomFilter = BloomFilter.create(TimeDimTupleFunnel.INSTANCE, 10);
Assert.assertFalse(bloomFilter.mightContain(tuple));
bloomFilter.put(tuple);
Assert.assertTrue(bloomFilter.mightContain(tuple));
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.distribution;
import org.hamcrest.Matchers;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TimeDimTupleTest
{
private static final long TIMESTAMP = 1000;
private static final String DIMENSION1 = "a";
private static final String DIMENSION2 = "m";
private static final String DIMENSION3 = "z";
private TimeDimTuple target;
@Before
public void setup()
{
target = new TimeDimTuple(TIMESTAMP, DIMENSION2);
}
@Test
public void comparesCorrectlyToSmallerTimestamp()
{
Assert.assertThat(target.compareTo(new TimeDimTuple(TIMESTAMP - 1, DIMENSION2)), Matchers.greaterThan(0));
}
@Test
public void comparesCorrectlyToSmallerDimension()
{
Assert.assertThat(target.compareTo(new TimeDimTuple(TIMESTAMP, DIMENSION1)), Matchers.greaterThan(0));
}
@Test
public void comparesCorrectlyToEqual()
{
Assert.assertEquals(0, target.compareTo(new TimeDimTuple(TIMESTAMP, DIMENSION2)));
}
@Test
public void comparesCorrectlyToBiggerTimestamp()
{
Assert.assertThat(target.compareTo(new TimeDimTuple(TIMESTAMP + 1, DIMENSION2)), Matchers.lessThan(0));
}
@Test
public void comparesCorrectlyToBiggerDimension()
{
Assert.assertThat(target.compareTo(new TimeDimTuple(TIMESTAMP, DIMENSION3)), Matchers.lessThan(0));
}
}

View File

@ -22,6 +22,7 @@ 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.data.input.MapBasedInputRow;
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;
@ -52,11 +53,11 @@ class IndexTaskInputRowIteratorBuilderTestingFactory
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;
return new MapBasedInputRow(
timestamp,
dimensionValues,
Collections.singletonMap(DIMENSION, dimensionValues)
);
}
static CloseableIterator<InputRow> createInputRowIterator(InputRow inputRow)
@ -75,6 +76,7 @@ class IndexTaskInputRowIteratorBuilderTestingFactory
return true;
}
@SuppressWarnings("IteratorNextCanNotThrowNoSuchElementException")
@Override
public InputRow next()
{

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.iterator;
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 org.hamcrest.Matchers;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
public class RangePartitionTaskInputRowIteratorBuilderTest
{
private static final boolean SKIP_NULL = true;
private static final IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester HANDLER_TESTER =
IndexTaskInputRowIteratorBuilderTestingFactory.createHandlerTester(
() -> new RangePartitionIndexTaskInputRowIteratorBuilder(
IndexTaskInputRowIteratorBuilderTestingFactory.DIMENSION,
SKIP_NULL
)
);
private static final InputRow NO_NEXT_INPUT_ROW = null;
@Test
public void invokesDimensionValueCountFilterLast()
{
DateTime timestamp = IndexTaskInputRowIteratorBuilderTestingFactory.TIMESTAMP;
List<String> multipleDimensionValues = Arrays.asList("multiple", "dimension", "values");
InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(
timestamp,
multipleDimensionValues
);
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,
NO_NEXT_INPUT_ROW
);
assertNotInHandlerInvocationHistory(
handlerInvocationHistory,
IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.NULL_ROW
);
assertNotInHandlerInvocationHistory(
handlerInvocationHistory,
IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.ABSENT_BUCKET_INTERVAL
);
}
@Test
public void doesNotInvokeHandlersIfRowValid()
{
DateTime timestamp = IndexTaskInputRowIteratorBuilderTestingFactory.TIMESTAMP;
List<String> nullDimensionValue = Collections.singletonList(null);
InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(timestamp, nullDimensionValue);
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);
}
@Test
public void invokesHandlerIfRowInvalidNull()
{
DateTime timestamp = IndexTaskInputRowIteratorBuilderTestingFactory.TIMESTAMP;
List<String> nullDimensionValue = null;
InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(timestamp, nullDimensionValue);
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,
NO_NEXT_INPUT_ROW
);
assertNotInHandlerInvocationHistory(
handlerInvocationHistory,
IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.NULL_ROW
);
assertNotInHandlerInvocationHistory(
handlerInvocationHistory,
IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler.ABSENT_BUCKET_INTERVAL
);
}
@Test
public void doesNotInvokeHandlersIfRowValidNull()
{
DateTime timestamp = IndexTaskInputRowIteratorBuilderTestingFactory.TIMESTAMP;
List<String> nullDimensionValue = null;
InputRow inputRow = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRow(timestamp, nullDimensionValue);
CloseableIterator<InputRow> inputRowIterator = IndexTaskInputRowIteratorBuilderTestingFactory.createInputRowIterator(
inputRow
);
GranularitySpec granularitySpec = IndexTaskInputRowIteratorBuilderTestingFactory.createGranularitySpec(
timestamp,
IndexTaskInputRowIteratorBuilderTestingFactory.PRESENT_BUCKET_INTERVAL_OPT
);
IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester handlerTester =
IndexTaskInputRowIteratorBuilderTestingFactory.createHandlerTester(
() -> new RangePartitionIndexTaskInputRowIteratorBuilder(
IndexTaskInputRowIteratorBuilderTestingFactory.DIMENSION,
!SKIP_NULL
)
);
List<IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler> handlerInvocationHistory =
handlerTester.invokeHandlers(
inputRowIterator,
granularitySpec,
inputRow
);
Assert.assertEquals(Collections.emptyList(), handlerInvocationHistory);
}
private static void assertNotInHandlerInvocationHistory(
List<IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler> handlerInvocationHistory,
IndexTaskInputRowIteratorBuilderTestingFactory.HandlerTester.Handler handler
)
{
Assert.assertThat(handlerInvocationHistory, Matchers.not(Matchers.contains(handler)));
}
}

View File

@ -29,6 +29,7 @@ public class TestNGGroup
public static final String HADOOP_INDEX = "hadoop-index";
public static final String KAFKA_INDEX = "kafka-index";
public static final String OTHER_INDEX = "other-index";
public static final String PERFECT_ROLLUP_PARALLEL_BATCH_INDEX = "perfect-rollup-parallel-batch-index";
public static final String QUERY = "query";
public static final String REALTIME_INDEX = "realtime-index";
public static final String SECURITY = "security";

View File

@ -21,8 +21,11 @@ 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.PartialDimensionDistributionTask;
import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask;
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.PartialRangeSegmentGenerateTask;
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;
@ -223,7 +226,7 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
);
}
// ITParallelIndexTest does a second round of ingestion to replace segements in an existing
// IT*ParallelIndexTest do a second round of ingestion to replace segements in an existing
// data source. For that second round we need to make sure the coordinator actually learned
// about the new segments befor waiting for it to report that all segments are loaded; otherwise
// this method could return too early because the coordinator is merely reporting that all the
@ -260,7 +263,10 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
return t.getType().equals(SinglePhaseSubTask.TYPE);
} else {
return t.getType().equalsIgnoreCase(PartialHashSegmentGenerateTask.TYPE)
|| t.getType().equalsIgnoreCase(PartialHashSegmentMergeTask.TYPE);
|| t.getType().equalsIgnoreCase(PartialHashSegmentMergeTask.TYPE)
|| t.getType().equalsIgnoreCase(PartialDimensionDistributionTask.TYPE)
|| t.getType().equalsIgnoreCase(PartialRangeSegmentGenerateTask.TYPE)
|| t.getType().equalsIgnoreCase(PartialGenericSegmentMergeTask.TYPE);
}
})
.count();

View File

@ -21,11 +21,11 @@ package org.apache.druid.tests.indexer;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.testing.guice.DruidTestModuleFactory;
import org.apache.druid.tests.TestNGGroup;
import org.testng.Assert;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
@ -35,7 +35,7 @@ import java.util.function.Function;
@Test(groups = TestNGGroup.BATCH_INDEX)
@Guice(moduleFactory = DruidTestModuleFactory.class)
public class ITParallelIndexTest extends AbstractITBatchIndexTest
public class ITBestEffortRollupParallelIndexTest extends AbstractITBatchIndexTest
{
private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json";
private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json";
@ -51,26 +51,27 @@ public class ITParallelIndexTest extends AbstractITBatchIndexTest
public static Object[][] resources()
{
return new Object[][]{
{new DynamicPartitionsSpec(null, null)},
{new HashedPartitionsSpec(null, 2, null)}
{new DynamicPartitionsSpec(null, null)}
};
}
@Test(dataProvider = "resources")
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 Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix())
try (
final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
final Closeable ignored2 = unloader(INDEX_INGEST_SEGMENT_DATASOURCE + config.getExtraDatasourceNameSuffix());
final Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix())
) {
boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible();
Assert.assertFalse(forceGuaranteedRollup, "parititionSpec does not support best-effort rollup");
final Function<String, String> rollupTransform = spec -> {
try {
spec = StringUtils.replace(
spec,
"%%FORCE_GUARANTEED_ROLLUP%%",
Boolean.toString(forceGuaranteedRollup)
Boolean.toString(false)
);
return StringUtils.replace(
spec,
@ -91,52 +92,32 @@ public class ITParallelIndexTest extends AbstractITBatchIndexTest
false
);
// Missing intervals is not supported yet if forceGuaranteedRollup = true
if (!forceGuaranteedRollup) {
// Index again, this time only choosing the second data file, and without explicit intervals chosen.
// The second datafile covers both day segments, so this should replace them, as reflected in the queries.
doIndexTest(
INDEX_DATASOURCE,
REINDEX_TASK,
rollupTransform,
REINDEX_QUERIES_RESOURCE,
true
);
// Index again, this time only choosing the second data file, and without explicit intervals chosen.
// The second datafile covers both day segments, so this should replace them, as reflected in the queries.
doIndexTest(
INDEX_DATASOURCE,
REINDEX_TASK,
rollupTransform,
REINDEX_QUERIES_RESOURCE,
true
);
doReindexTest(
INDEX_DATASOURCE,
INDEX_INGEST_SEGMENT_DATASOURCE,
rollupTransform,
INDEX_INGEST_SEGMENT_TASK,
REINDEX_QUERIES_RESOURCE
);
doReindexTest(
INDEX_DATASOURCE,
INDEX_INGEST_SEGMENT_DATASOURCE,
rollupTransform,
INDEX_INGEST_SEGMENT_TASK,
REINDEX_QUERIES_RESOURCE
);
// with DruidInputSource instead of IngestSegmentFirehose
doReindexTest(
INDEX_DATASOURCE,
INDEX_DRUID_INPUT_SOURCE_DATASOURCE,
rollupTransform,
INDEX_DRUID_INPUT_SOURCE_TASK,
REINDEX_QUERIES_RESOURCE
);
} else {
doReindexTest(
INDEX_DATASOURCE,
INDEX_INGEST_SEGMENT_DATASOURCE,
rollupTransform,
INDEX_INGEST_SEGMENT_TASK,
INDEX_QUERIES_RESOURCE
);
// with DruidInputSource instead of IngestSegmentFirehose
doReindexTest(
INDEX_DATASOURCE,
INDEX_DRUID_INPUT_SOURCE_DATASOURCE,
rollupTransform,
INDEX_DRUID_INPUT_SOURCE_TASK,
INDEX_QUERIES_RESOURCE
);
}
// with DruidInputSource instead of IngestSegmentFirehose
doReindexTest(
INDEX_DATASOURCE,
INDEX_DRUID_INPUT_SOURCE_DATASOURCE,
rollupTransform,
INDEX_DRUID_INPUT_SOURCE_TASK,
REINDEX_QUERIES_RESOURCE
);
}
}
}

View File

@ -0,0 +1,113 @@
/*
* 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.tests.indexer;
import com.fasterxml.jackson.core.JsonProcessingException;
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.java.util.common.StringUtils;
import org.apache.druid.testing.guice.DruidTestModuleFactory;
import org.apache.druid.tests.TestNGGroup;
import org.testng.Assert;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
import java.io.Closeable;
import java.util.function.Function;
@Test(groups = TestNGGroup.PERFECT_ROLLUP_PARALLEL_BATCH_INDEX)
@Guice(moduleFactory = DruidTestModuleFactory.class)
public class ITPerfectRollupParallelIndexTest extends AbstractITBatchIndexTest
{
private static final String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json";
private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json";
private static final String INDEX_DATASOURCE = "wikipedia_parallel_index_test";
private static final String INDEX_INGEST_SEGMENT_DATASOURCE = "wikipedia_parallel_ingest_segment_index_test";
private static final String INDEX_INGEST_SEGMENT_TASK = "/indexer/wikipedia_parallel_ingest_segment_index_task.json";
private static final String INDEX_DRUID_INPUT_SOURCE_DATASOURCE = "wikipedia_parallel_druid_input_source_index_test";
private static final String INDEX_DRUID_INPUT_SOURCE_TASK = "/indexer/wikipedia_parallel_druid_input_source_index_task.json";
@DataProvider
public static Object[][] resources()
{
return new Object[][]{
{new HashedPartitionsSpec(null, 2, null)},
{new SingleDimensionPartitionsSpec(2, null, "namespace", false)}
};
}
@Test(dataProvider = "resources")
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 Closeable ignored3 = unloader(INDEX_DRUID_INPUT_SOURCE_DATASOURCE + config.getExtraDatasourceNameSuffix())
) {
boolean forceGuaranteedRollup = partitionsSpec.isForceGuaranteedRollupCompatible();
Assert.assertTrue(forceGuaranteedRollup, "parititionSpec does not support perfect rollup");
final Function<String, String> rollupTransform = spec -> {
try {
spec = StringUtils.replace(
spec,
"%%FORCE_GUARANTEED_ROLLUP%%",
Boolean.toString(true)
);
return StringUtils.replace(
spec,
"%%PARTITIONS_SPEC%%",
jsonMapper.writeValueAsString(partitionsSpec)
);
}
catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
};
doIndexTest(
INDEX_DATASOURCE,
INDEX_TASK,
rollupTransform,
INDEX_QUERIES_RESOURCE,
false
);
doReindexTest(
INDEX_DATASOURCE,
INDEX_INGEST_SEGMENT_DATASOURCE,
rollupTransform,
INDEX_INGEST_SEGMENT_TASK,
INDEX_QUERIES_RESOURCE
);
// with DruidInputSource instead of IngestSegmentFirehose
doReindexTest(
INDEX_DATASOURCE,
INDEX_DRUID_INPUT_SOURCE_DATASOURCE,
rollupTransform,
INDEX_DRUID_INPUT_SOURCE_TASK,
INDEX_QUERIES_RESOURCE
);
}
}
}

10
pom.xml
View File

@ -992,6 +992,16 @@
<artifactId>api-util</artifactId>
<version>1.0.3</version>
</dependency>
<dependency>
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-java</artifactId>
<version>1.1.0-incubating</version>
</dependency>
<dependency>
<groupId>org.apache.datasketches</groupId>
<artifactId>datasketches-memory</artifactId>
<version>1.2.0-incubating</version>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>