mirror of https://github.com/apache/druid.git
Refactoring some codes around ingestion (#9274)
* Refactoring codes around ingestion: - Parallel index task and simple task now use the same segment allocator implementation. This is reusable for the future implementation as well. - Added PartitionAnalysis to store the analysis of the partitioning - Move some util methods to SegmentLockHelper and rename it to TaskLockHelper * fix build * fix SingleDimensionShardSpecFactory * optimize SingledimensionShardSpecFactory * fix test * shard spec builder * import order * shardSpecBuilder -> partialShardSpec * build -> complete * fix comment; add unit tests for partitionBoundaries * add more tests and fix javadoc * fix toString(); add serde tests for HashBasedNumberedPartialShardSpec and SegmentAllocateAction * fix test * add equality test for hash and range partial shard specs
This commit is contained in:
parent
b55657cc26
commit
e81230f9ab
|
@ -324,6 +324,11 @@
|
||||||
<artifactId>jetty-util</artifactId>
|
<artifactId>jetty-util</artifactId>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>nl.jqno.equalsverifier</groupId>
|
||||||
|
<artifactId>equalsverifier</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.google.errorprone</groupId>
|
<groupId>com.google.errorprone</groupId>
|
||||||
<artifactId>error_prone_annotations</artifactId>
|
<artifactId>error_prone_annotations</artifactId>
|
||||||
|
|
|
@ -52,6 +52,12 @@ public class DynamicPartitionsSpec implements PartitionsSpec
|
||||||
this.maxTotalRows = maxTotalRows;
|
this.maxTotalRows = maxTotalRows;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SecondaryPartitionType getType()
|
||||||
|
{
|
||||||
|
return SecondaryPartitionType.LINEAR;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public Integer getMaxRowsPerSegment()
|
public Integer getMaxRowsPerSegment()
|
||||||
|
|
|
@ -123,6 +123,12 @@ public class HashedPartitionsSpec implements DimensionBasedPartitionsSpec
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SecondaryPartitionType getType()
|
||||||
|
{
|
||||||
|
return SecondaryPartitionType.HASH;
|
||||||
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
|
|
@ -41,6 +41,9 @@ public interface PartitionsSpec
|
||||||
String MAX_ROWS_PER_SEGMENT = "maxRowsPerSegment";
|
String MAX_ROWS_PER_SEGMENT = "maxRowsPerSegment";
|
||||||
int HISTORICAL_NULL = -1;
|
int HISTORICAL_NULL = -1;
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
SecondaryPartitionType getType();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the max number of rows per segment.
|
* Returns the max number of rows per segment.
|
||||||
* Implementations can have different default values which it could be even null.
|
* Implementations can have different default values which it could be even null.
|
||||||
|
@ -55,28 +58,6 @@ public interface PartitionsSpec
|
||||||
*/
|
*/
|
||||||
boolean needsDeterminePartitions(boolean useForHadoopTask);
|
boolean needsDeterminePartitions(boolean useForHadoopTask);
|
||||||
|
|
||||||
/**
|
|
||||||
* '-1' regarded as null for some historical reason.
|
|
||||||
*/
|
|
||||||
static boolean isEffectivelyNull(@Nullable Integer val)
|
|
||||||
{
|
|
||||||
return val == null || val == HISTORICAL_NULL;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* '-1' regarded as null for some historical reason.
|
|
||||||
*/
|
|
||||||
static boolean isEffectivelyNull(@Nullable Long val)
|
|
||||||
{
|
|
||||||
return val == null || val == HISTORICAL_NULL;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
static Integer resolveHistoricalNullIfNeeded(@Nullable Integer val)
|
|
||||||
{
|
|
||||||
return isEffectivelyNull(val) ? null : val;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return True if this partitionSpec's type is compatible with forceGuaranteedRollup=true.
|
* @return True if this partitionSpec's type is compatible with forceGuaranteedRollup=true.
|
||||||
*/
|
*/
|
||||||
|
@ -101,4 +82,26 @@ public interface PartitionsSpec
|
||||||
*/
|
*/
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
String getForceGuaranteedRollupIncompatiblityReason();
|
String getForceGuaranteedRollupIncompatiblityReason();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* '-1' regarded as null for some historical reason.
|
||||||
|
*/
|
||||||
|
static boolean isEffectivelyNull(@Nullable Integer val)
|
||||||
|
{
|
||||||
|
return val == null || val == HISTORICAL_NULL;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* '-1' regarded as null for some historical reason.
|
||||||
|
*/
|
||||||
|
static boolean isEffectivelyNull(@Nullable Long val)
|
||||||
|
{
|
||||||
|
return val == null || val == HISTORICAL_NULL;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
static Integer resolveHistoricalNullIfNeeded(@Nullable Integer val)
|
||||||
|
{
|
||||||
|
return isEffectivelyNull(val) ? null : val;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,50 @@
|
||||||
|
/*
|
||||||
|
* 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.indexer.partitions;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* In Druid, ingested data are primarily partitioned based on time range (@link GranularitySpec#getSegmentGranularity),
|
||||||
|
* and then secondly partitioned based on the given {@link PartitionsSpec}. This enum lists all supported types for the
|
||||||
|
* secondary partitioning.
|
||||||
|
*/
|
||||||
|
public enum SecondaryPartitionType
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* Linear partitioning partitions segments in the same time chunk based on their size or number of rows in them.
|
||||||
|
*
|
||||||
|
* @see DynamicPartitionsSpec
|
||||||
|
* @see org.apache.druid.timeline.partition.NumberedShardSpec
|
||||||
|
*/
|
||||||
|
LINEAR,
|
||||||
|
/**
|
||||||
|
* Hash partitioning partitions segments in the same time chunk based on the hash value of the partition dimensions.
|
||||||
|
*
|
||||||
|
* @see HashedPartitionsSpec
|
||||||
|
* @see org.apache.druid.timeline.partition.HashBasedNumberedShardSpec
|
||||||
|
*/
|
||||||
|
HASH,
|
||||||
|
/**
|
||||||
|
* Range partitioning partitions segments in the same time chunk based on the value range of the partition dimension.
|
||||||
|
*
|
||||||
|
* @see SingleDimensionPartitionsSpec
|
||||||
|
* @see org.apache.druid.timeline.partition.SingleDimensionShardSpec
|
||||||
|
*/
|
||||||
|
RANGE
|
||||||
|
}
|
|
@ -138,6 +138,12 @@ public class SingleDimensionPartitionsSpec implements DimensionBasedPartitionsSp
|
||||||
return targetRowsPerSegment;
|
return targetRowsPerSegment;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SecondaryPartitionType getType()
|
||||||
|
{
|
||||||
|
return SecondaryPartitionType.RANGE;
|
||||||
|
}
|
||||||
|
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
@Override
|
@Override
|
||||||
@NotNull
|
@NotNull
|
||||||
|
|
|
@ -27,20 +27,22 @@ import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
public class HashBasedNumberedShardSpecFactory implements ShardSpecFactory
|
public class HashBasedNumberedPartialShardSpec implements PartialShardSpec
|
||||||
{
|
{
|
||||||
|
public static final String TYPE = "hashed";
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
private final List<String> partitionDimensions;
|
private final List<String> partitionDimensions;
|
||||||
private final int numPartitions;
|
private final int numBuckets;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public HashBasedNumberedShardSpecFactory(
|
public HashBasedNumberedPartialShardSpec(
|
||||||
@JsonProperty("partitionDimensions") @Nullable List<String> partitionDimensions,
|
@JsonProperty("partitionDimensions") @Nullable List<String> partitionDimensions,
|
||||||
@JsonProperty("numPartitions") int numPartitions
|
@JsonProperty("numPartitions") int numBuckets
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.partitionDimensions = partitionDimensions;
|
this.partitionDimensions = partitionDimensions;
|
||||||
this.numPartitions = numPartitions;
|
this.numBuckets = numBuckets;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -50,27 +52,28 @@ public class HashBasedNumberedShardSpecFactory implements ShardSpecFactory
|
||||||
return partitionDimensions;
|
return partitionDimensions;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty public int getNumPartitions()
|
@JsonProperty("numPartitions")
|
||||||
|
public int getNumBuckets()
|
||||||
{
|
{
|
||||||
return numPartitions;
|
return numBuckets;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
||||||
{
|
{
|
||||||
final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId;
|
final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId;
|
||||||
return new HashBasedNumberedShardSpec(
|
return new HashBasedNumberedShardSpec(
|
||||||
prevSpec == null ? 0 : prevSpec.getPartitionNum() + 1,
|
prevSpec == null ? 0 : prevSpec.getPartitionNum() + 1,
|
||||||
numPartitions,
|
numBuckets,
|
||||||
partitionDimensions,
|
partitionDimensions,
|
||||||
objectMapper
|
objectMapper
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, int partitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
|
||||||
{
|
{
|
||||||
return new HashBasedNumberedShardSpec(partitionId, numPartitions, partitionDimensions, objectMapper);
|
return new HashBasedNumberedShardSpec(partitionId, numBuckets, partitionDimensions, objectMapper);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -88,14 +91,14 @@ public class HashBasedNumberedShardSpecFactory implements ShardSpecFactory
|
||||||
if (o == null || getClass() != o.getClass()) {
|
if (o == null || getClass() != o.getClass()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
HashBasedNumberedShardSpecFactory that = (HashBasedNumberedShardSpecFactory) o;
|
HashBasedNumberedPartialShardSpec that = (HashBasedNumberedPartialShardSpec) o;
|
||||||
return numPartitions == that.numPartitions &&
|
return numBuckets == that.numBuckets &&
|
||||||
Objects.equals(partitionDimensions, that.partitionDimensions);
|
Objects.equals(partitionDimensions, that.partitionDimensions);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
return Objects.hash(partitionDimensions, numPartitions);
|
return Objects.hash(partitionDimensions, numBuckets);
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -23,21 +23,21 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
public class LinearShardSpecFactory implements ShardSpecFactory
|
public class LinearPartialShardSpec implements PartialShardSpec
|
||||||
{
|
{
|
||||||
private static final LinearShardSpecFactory INSTANCE = new LinearShardSpecFactory();
|
private static final LinearPartialShardSpec INSTANCE = new LinearPartialShardSpec();
|
||||||
|
|
||||||
public static LinearShardSpecFactory instance()
|
public static LinearPartialShardSpec instance()
|
||||||
{
|
{
|
||||||
return INSTANCE;
|
return INSTANCE;
|
||||||
}
|
}
|
||||||
|
|
||||||
private LinearShardSpecFactory()
|
private LinearPartialShardSpec()
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
||||||
{
|
{
|
||||||
return new LinearShardSpec(
|
return new LinearShardSpec(
|
||||||
specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1
|
specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1
|
||||||
|
@ -45,7 +45,7 @@ public class LinearShardSpecFactory implements ShardSpecFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, int partitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
|
||||||
{
|
{
|
||||||
return new LinearShardSpec(partitionId);
|
return new LinearShardSpec(partitionId);
|
||||||
}
|
}
|
|
@ -25,14 +25,14 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
public class NumberedOverwritingShardSpecFactory implements ShardSpecFactory
|
public class NumberedOverwritePartialShardSpec implements PartialShardSpec
|
||||||
{
|
{
|
||||||
private final int startRootPartitionId;
|
private final int startRootPartitionId;
|
||||||
private final int endRootPartitionId;
|
private final int endRootPartitionId;
|
||||||
private final short minorVersion;
|
private final short minorVersion;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public NumberedOverwritingShardSpecFactory(
|
public NumberedOverwritePartialShardSpec(
|
||||||
@JsonProperty("startRootPartitionId") int startRootPartitionId,
|
@JsonProperty("startRootPartitionId") int startRootPartitionId,
|
||||||
@JsonProperty("endRootPartitionId") int endRootPartitionId,
|
@JsonProperty("endRootPartitionId") int endRootPartitionId,
|
||||||
@JsonProperty("minorVersion") short minorVersion
|
@JsonProperty("minorVersion") short minorVersion
|
||||||
|
@ -62,7 +62,7 @@ public class NumberedOverwritingShardSpecFactory implements ShardSpecFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
||||||
{
|
{
|
||||||
// specOfPreviousMaxPartitionId is the max partitionId of the same shardSpec
|
// specOfPreviousMaxPartitionId is the max partitionId of the same shardSpec
|
||||||
// and could be null if all existing segments are first-generation segments.
|
// and could be null if all existing segments are first-generation segments.
|
||||||
|
@ -77,7 +77,7 @@ public class NumberedOverwritingShardSpecFactory implements ShardSpecFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, int partitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
|
||||||
{
|
{
|
||||||
return new NumberedOverwriteShardSpec(partitionId, startRootPartitionId, endRootPartitionId, minorVersion);
|
return new NumberedOverwriteShardSpec(partitionId, startRootPartitionId, endRootPartitionId, minorVersion);
|
||||||
}
|
}
|
|
@ -23,21 +23,21 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
public class NumberedShardSpecFactory implements ShardSpecFactory
|
public class NumberedPartialShardSpec implements PartialShardSpec
|
||||||
{
|
{
|
||||||
private static final NumberedShardSpecFactory INSTANCE = new NumberedShardSpecFactory();
|
private static final NumberedPartialShardSpec INSTANCE = new NumberedPartialShardSpec();
|
||||||
|
|
||||||
public static NumberedShardSpecFactory instance()
|
public static NumberedPartialShardSpec instance()
|
||||||
{
|
{
|
||||||
return INSTANCE;
|
return INSTANCE;
|
||||||
}
|
}
|
||||||
|
|
||||||
private NumberedShardSpecFactory()
|
private NumberedPartialShardSpec()
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
||||||
{
|
{
|
||||||
if (specOfPreviousMaxPartitionId == null) {
|
if (specOfPreviousMaxPartitionId == null) {
|
||||||
return new NumberedShardSpec(0, 0);
|
return new NumberedShardSpec(0, 0);
|
||||||
|
@ -48,7 +48,7 @@ public class NumberedShardSpecFactory implements ShardSpecFactory
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ShardSpec create(ObjectMapper objectMapper, int partitionId)
|
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
|
||||||
{
|
{
|
||||||
return new NumberedShardSpec(partitionId, 0);
|
return new NumberedShardSpec(partitionId, 0);
|
||||||
}
|
}
|
|
@ -19,38 +19,43 @@
|
||||||
|
|
||||||
package org.apache.druid.timeline.partition;
|
package org.apache.druid.timeline.partition;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Factory to be used to allocate segments remotely in the overlord.
|
* Class to contain all information of a {@link ShardSpec} except for the partition ID.
|
||||||
|
* This class is mainly used by the indexing tasks to allocate new segments using the Overlord.
|
||||||
*/
|
*/
|
||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||||
@JsonSubTypes({
|
@JsonSubTypes({
|
||||||
@JsonSubTypes.Type(name = "numbered", value = NumberedShardSpecFactory.class),
|
@Type(name = "numbered", value = NumberedPartialShardSpec.class),
|
||||||
@JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpecFactory.class),
|
@Type(name = HashBasedNumberedPartialShardSpec.TYPE, value = HashBasedNumberedPartialShardSpec.class),
|
||||||
@JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwritingShardSpecFactory.class),
|
@Type(name = "single_dim", value = SingleDimensionPartialShardSpec.class),
|
||||||
|
@Type(name = "numbered_overwrite", value = NumberedOverwritePartialShardSpec.class),
|
||||||
})
|
})
|
||||||
public interface ShardSpecFactory
|
public interface PartialShardSpec
|
||||||
{
|
{
|
||||||
/**
|
/**
|
||||||
* Create a new shardSpec based on {@code specOfPreviousMaxPartitionId}. If it's null, it assumes that this is the
|
* Creates a new ShardSpec based on {@code specOfPreviousMaxPartitionId}. If it's null, it assumes that this is the
|
||||||
* first call for the timeChunk where the new segment is created.
|
* first call for the time chunk where the new segment is created.
|
||||||
* Note that {@code specOfPreviousMaxPartitionId} can also be null for {@link OverwriteShardSpec} if all segments
|
* Note that {@code specOfPreviousMaxPartitionId} can also be null for {@link OverwriteShardSpec} if all segments
|
||||||
* in the timeChunk are first-generation segments.
|
* in the timeChunk are first-generation segments.
|
||||||
*/
|
*/
|
||||||
ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId);
|
ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new shardSpec having the given partitionId.
|
* Creates a new shardSpec having the given partitionId.
|
||||||
*/
|
*/
|
||||||
ShardSpec create(ObjectMapper objectMapper, int partitionId);
|
ShardSpec complete(ObjectMapper objectMapper, int partitionId);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the class of the shardSpec created by this factory.
|
* Returns the class of the shardSpec created by this factory.
|
||||||
*/
|
*/
|
||||||
|
@JsonIgnore
|
||||||
Class<? extends ShardSpec> getShardSpecClass();
|
Class<? extends ShardSpec> getShardSpecClass();
|
||||||
}
|
}
|
|
@ -17,7 +17,7 @@
|
||||||
* under the License.
|
* under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.task.batch.parallel.distribution;
|
package org.apache.druid.timeline.partition;
|
||||||
|
|
||||||
import com.google.common.collect.ForwardingList;
|
import com.google.common.collect.ForwardingList;
|
||||||
|
|
||||||
|
@ -25,6 +25,7 @@ import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -74,4 +75,31 @@ public class PartitionBoundaries extends ForwardingList<String> implements List<
|
||||||
{
|
{
|
||||||
return delegate;
|
return delegate;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int getNumBuckets()
|
||||||
|
{
|
||||||
|
return delegate.size() - 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
@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;
|
||||||
|
}
|
||||||
|
PartitionBoundaries strings = (PartitionBoundaries) o;
|
||||||
|
return Objects.equals(delegate, strings.delegate);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(super.hashCode(), delegate);
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -0,0 +1,141 @@
|
||||||
|
/*
|
||||||
|
* 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.timeline.partition;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
public class SingleDimensionPartialShardSpec implements PartialShardSpec
|
||||||
|
{
|
||||||
|
private final String partitionDimension;
|
||||||
|
private final int bucketId;
|
||||||
|
@Nullable
|
||||||
|
private final String start;
|
||||||
|
@Nullable
|
||||||
|
private final String end;
|
||||||
|
private final int numBuckets;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public SingleDimensionPartialShardSpec(
|
||||||
|
@JsonProperty("partitionDimension") String partitionDimension,
|
||||||
|
@JsonProperty("bucketId") int bucketId,
|
||||||
|
@JsonProperty("start") @Nullable String start,
|
||||||
|
@JsonProperty("end") @Nullable String end,
|
||||||
|
@JsonProperty("numBuckets") int numBuckets
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.partitionDimension = partitionDimension;
|
||||||
|
this.bucketId = bucketId;
|
||||||
|
this.start = start;
|
||||||
|
this.end = end;
|
||||||
|
this.numBuckets = numBuckets;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getPartitionDimension()
|
||||||
|
{
|
||||||
|
return partitionDimension;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getBucketId()
|
||||||
|
{
|
||||||
|
return bucketId;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@Nullable
|
||||||
|
public String getStart()
|
||||||
|
{
|
||||||
|
return start;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@Nullable
|
||||||
|
public String getEnd()
|
||||||
|
{
|
||||||
|
return end;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getNumBuckets()
|
||||||
|
{
|
||||||
|
return numBuckets;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
|
||||||
|
{
|
||||||
|
final int partitionId;
|
||||||
|
if (specOfPreviousMaxPartitionId != null) {
|
||||||
|
assert specOfPreviousMaxPartitionId instanceof SingleDimensionShardSpec;
|
||||||
|
final SingleDimensionShardSpec prevSpec = (SingleDimensionShardSpec) specOfPreviousMaxPartitionId;
|
||||||
|
partitionId = prevSpec.getPartitionNum() + 1;
|
||||||
|
} else {
|
||||||
|
partitionId = 0;
|
||||||
|
}
|
||||||
|
return complete(objectMapper, partitionId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
|
||||||
|
{
|
||||||
|
// TODO: bucketId and numBuckets should be added to SingleDimensionShardSpec in a follow-up PR.
|
||||||
|
return new SingleDimensionShardSpec(
|
||||||
|
partitionDimension,
|
||||||
|
start,
|
||||||
|
end,
|
||||||
|
partitionId
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<? extends ShardSpec> getShardSpecClass()
|
||||||
|
{
|
||||||
|
return SingleDimensionShardSpec.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
SingleDimensionPartialShardSpec that = (SingleDimensionPartialShardSpec) o;
|
||||||
|
return bucketId == that.bucketId &&
|
||||||
|
numBuckets == that.numBuckets &&
|
||||||
|
Objects.equals(partitionDimension, that.partitionDimension) &&
|
||||||
|
Objects.equals(start, that.start) &&
|
||||||
|
Objects.equals(end, that.end);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(partitionDimension, bucketId, start, end, numBuckets);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,74 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.druid.timeline.partition;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class HashBasedNumberedPartialShardSpecTest
|
||||||
|
{
|
||||||
|
private static final ObjectMapper MAPPER = new ObjectMapper();
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEquals()
|
||||||
|
{
|
||||||
|
EqualsVerifier.forClass(HashBasedNumberedPartialShardSpec.class)
|
||||||
|
.usingGetClass()
|
||||||
|
.withNonnullFields("partitionDimensions", "numBuckets")
|
||||||
|
.verify();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSerde() throws IOException
|
||||||
|
{
|
||||||
|
final HashBasedNumberedPartialShardSpec expected = new HashBasedNumberedPartialShardSpec(
|
||||||
|
ImmutableList.of("dim1", "dim2"),
|
||||||
|
3
|
||||||
|
);
|
||||||
|
final byte[] json = MAPPER.writeValueAsBytes(expected);
|
||||||
|
final HashBasedNumberedPartialShardSpec fromJson = (HashBasedNumberedPartialShardSpec) MAPPER.readValue(
|
||||||
|
json,
|
||||||
|
PartialShardSpec.class
|
||||||
|
);
|
||||||
|
Assert.assertEquals(expected, fromJson);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testJsonPropertyNames() throws IOException
|
||||||
|
{
|
||||||
|
final HashBasedNumberedPartialShardSpec expected = new HashBasedNumberedPartialShardSpec(
|
||||||
|
ImmutableList.of("dim1", "dim2"),
|
||||||
|
3
|
||||||
|
);
|
||||||
|
final byte[] json = MAPPER.writeValueAsBytes(expected);
|
||||||
|
//noinspection unchecked
|
||||||
|
final Map<String, Object> map = MAPPER.readValue(json, Map.class);
|
||||||
|
Assert.assertEquals(3, map.size());
|
||||||
|
Assert.assertEquals(HashBasedNumberedPartialShardSpec.TYPE, map.get("type"));
|
||||||
|
Assert.assertEquals(expected.getPartitionDimensions(), map.get("partitionDimensions"));
|
||||||
|
Assert.assertEquals(expected.getNumBuckets(), map.get("numPartitions"));
|
||||||
|
}
|
||||||
|
}
|
|
@ -17,9 +17,11 @@
|
||||||
* under the License.
|
* under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.task.batch.parallel.distribution;
|
package org.apache.druid.timeline.partition;
|
||||||
|
|
||||||
import org.apache.druid.segment.TestHelper;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
@ -74,8 +76,23 @@ public class PartitionBoundariesTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void serializesDeserializes()
|
public void serializesDeserializes() throws JsonProcessingException
|
||||||
{
|
{
|
||||||
TestHelper.testSerializesDeserializes(TestHelper.JSON_MAPPER, target);
|
final ObjectMapper objectMapper = new ObjectMapper();
|
||||||
|
String serialized = objectMapper.writeValueAsString(target);
|
||||||
|
Object deserialized = objectMapper.readValue(serialized, target.getClass());
|
||||||
|
Assert.assertEquals(serialized, objectMapper.writeValueAsString(deserialized));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetNumBucketsOfNonEmptyPartitionBoundariesReturningCorrectSize()
|
||||||
|
{
|
||||||
|
Assert.assertEquals(2, target.getNumBuckets());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testEqualsContract()
|
||||||
|
{
|
||||||
|
EqualsVerifier.forClass(PartitionBoundaries.class).withNonnullFields("delegate").usingGetClass().verify();
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -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.timeline.partition;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public class SingleDimensionPartialShardSpecTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testEquals()
|
||||||
|
{
|
||||||
|
EqualsVerifier.forClass(SingleDimensionPartialShardSpec.class)
|
||||||
|
.usingGetClass()
|
||||||
|
.withNonnullFields("partitionDimension", "bucketId", "numBuckets")
|
||||||
|
.verify();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSerde() throws IOException
|
||||||
|
{
|
||||||
|
final SingleDimensionPartialShardSpec expected = new SingleDimensionPartialShardSpec(
|
||||||
|
"partitionKey",
|
||||||
|
3,
|
||||||
|
"start",
|
||||||
|
"end",
|
||||||
|
10
|
||||||
|
);
|
||||||
|
final ObjectMapper mapper = new ObjectMapper();
|
||||||
|
final byte[] json = mapper.writeValueAsBytes(expected);
|
||||||
|
final SingleDimensionPartialShardSpec fromJson = (SingleDimensionPartialShardSpec) mapper.readValue(
|
||||||
|
json,
|
||||||
|
PartialShardSpec.class
|
||||||
|
);
|
||||||
|
Assert.assertEquals(expected, fromJson);
|
||||||
|
}
|
||||||
|
}
|
|
@ -369,7 +369,14 @@ public class IndexGeneratorJobTest
|
||||||
new StringInputRowParser(
|
new StringInputRowParser(
|
||||||
new JSONParseSpec(
|
new JSONParseSpec(
|
||||||
new TimestampSpec("ts", "yyyyMMddHH", null),
|
new TimestampSpec("ts", "yyyyMMddHH", null),
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("B", "F", "M", "Q", "X", "Y")), null, null),
|
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of(
|
||||||
|
"B",
|
||||||
|
"F",
|
||||||
|
"M",
|
||||||
|
"Q",
|
||||||
|
"X",
|
||||||
|
"Y"
|
||||||
|
)), null, null),
|
||||||
null,
|
null,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
@ -553,12 +560,22 @@ public class IndexGeneratorJobTest
|
||||||
List<ShardSpec> specs = new ArrayList<>();
|
List<ShardSpec> specs = new ArrayList<>();
|
||||||
if ("hashed".equals(partitionType)) {
|
if ("hashed".equals(partitionType)) {
|
||||||
for (Integer[] shardInfo : (Integer[][]) shardInfoForEachShard) {
|
for (Integer[] shardInfo : (Integer[][]) shardInfoForEachShard) {
|
||||||
specs.add(new HashBasedNumberedShardSpec(shardInfo[0], shardInfo[1], null, HadoopDruidIndexerConfig.JSON_MAPPER));
|
specs.add(new HashBasedNumberedShardSpec(
|
||||||
|
shardInfo[0],
|
||||||
|
shardInfo[1],
|
||||||
|
null,
|
||||||
|
HadoopDruidIndexerConfig.JSON_MAPPER
|
||||||
|
));
|
||||||
}
|
}
|
||||||
} else if ("single".equals(partitionType)) {
|
} else if ("single".equals(partitionType)) {
|
||||||
int partitionNum = 0;
|
int partitionNum = 0;
|
||||||
for (String[] shardInfo : (String[][]) shardInfoForEachShard) {
|
for (String[] shardInfo : (String[][]) shardInfoForEachShard) {
|
||||||
specs.add(new SingleDimensionShardSpec("host", shardInfo[0], shardInfo[1], partitionNum++));
|
specs.add(new SingleDimensionShardSpec(
|
||||||
|
"host",
|
||||||
|
shardInfo[0],
|
||||||
|
shardInfo[1],
|
||||||
|
partitionNum++
|
||||||
|
));
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
throw new RE("Invalid partition type:[%s]", partitionType);
|
throw new RE("Invalid partition type:[%s]", partitionType);
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
public class ActionBasedSegmentAllocator implements SegmentAllocator
|
public class ActionBasedSegmentAllocator implements SegmentAllocator
|
||||||
|
@ -44,6 +45,7 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
|
||||||
this.actionGenerator = actionGenerator;
|
this.actionGenerator = actionGenerator;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
@Override
|
@Override
|
||||||
public SegmentIdWithShardSpec allocate(
|
public SegmentIdWithShardSpec allocate(
|
||||||
final InputRow row,
|
final InputRow row,
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.actions;
|
package org.apache.druid.indexing.common.actions;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.overlord.TaskStorage;
|
import org.apache.druid.indexing.overlord.TaskStorage;
|
||||||
|
@ -27,15 +26,10 @@ import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||||
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
|
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||||
|
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
|
|
||||||
public class LocalTaskActionClient implements TaskActionClient
|
public class LocalTaskActionClient implements TaskActionClient
|
||||||
{
|
{
|
||||||
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
|
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
|
||||||
|
|
||||||
private final ConcurrentHashMap<Class<? extends TaskAction>, AtomicInteger> actionCountMap = new ConcurrentHashMap<>();
|
|
||||||
|
|
||||||
private final Task task;
|
private final Task task;
|
||||||
private final TaskStorage storage;
|
private final TaskStorage storage;
|
||||||
private final TaskActionToolbox toolbox;
|
private final TaskActionToolbox toolbox;
|
||||||
|
@ -79,17 +73,9 @@ public class LocalTaskActionClient implements TaskActionClient
|
||||||
final long performStartTime = System.currentTimeMillis();
|
final long performStartTime = System.currentTimeMillis();
|
||||||
final RetType result = taskAction.perform(task, toolbox);
|
final RetType result = taskAction.perform(task, toolbox);
|
||||||
emitTimerMetric("task/action/run/time", System.currentTimeMillis() - performStartTime);
|
emitTimerMetric("task/action/run/time", System.currentTimeMillis() - performStartTime);
|
||||||
actionCountMap.computeIfAbsent(taskAction.getClass(), k -> new AtomicInteger()).incrementAndGet();
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
public int getActionCount(Class<? extends TaskAction> actionClass)
|
|
||||||
{
|
|
||||||
final AtomicInteger count = actionCountMap.get(actionClass);
|
|
||||||
return count == null ? 0 : count.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void emitTimerMetric(final String metric, final long time)
|
private void emitTimerMetric(final String metric, final long time)
|
||||||
{
|
{
|
||||||
final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder();
|
final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder();
|
||||||
|
|
|
@ -37,8 +37,8 @@ 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.logger.Logger;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
@ -63,6 +63,8 @@ import java.util.stream.Collectors;
|
||||||
*/
|
*/
|
||||||
public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
{
|
{
|
||||||
|
public static final String TYPE = "segmentAllocate";
|
||||||
|
|
||||||
private static final Logger log = new Logger(SegmentAllocateAction.class);
|
private static final Logger log = new Logger(SegmentAllocateAction.class);
|
||||||
|
|
||||||
// Prevent spinning forever in situations where the segment list just won't stop changing.
|
// Prevent spinning forever in situations where the segment list just won't stop changing.
|
||||||
|
@ -75,7 +77,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
private final String sequenceName;
|
private final String sequenceName;
|
||||||
private final String previousSegmentId;
|
private final String previousSegmentId;
|
||||||
private final boolean skipSegmentLineageCheck;
|
private final boolean skipSegmentLineageCheck;
|
||||||
private final ShardSpecFactory shardSpecFactory;
|
private final PartialShardSpec partialShardSpec;
|
||||||
private final LockGranularity lockGranularity;
|
private final LockGranularity lockGranularity;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
|
@ -88,7 +90,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
@JsonProperty("previousSegmentId") String previousSegmentId,
|
@JsonProperty("previousSegmentId") String previousSegmentId,
|
||||||
@JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck,
|
@JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck,
|
||||||
// nullable for backward compatibility
|
// nullable for backward compatibility
|
||||||
@JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory,
|
@JsonProperty("shardSpecFactory") @Nullable PartialShardSpec partialShardSpec,
|
||||||
@JsonProperty("lockGranularity") @Nullable LockGranularity lockGranularity // nullable for backward compatibility
|
@JsonProperty("lockGranularity") @Nullable LockGranularity lockGranularity // nullable for backward compatibility
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -102,7 +104,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName");
|
this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName");
|
||||||
this.previousSegmentId = previousSegmentId;
|
this.previousSegmentId = previousSegmentId;
|
||||||
this.skipSegmentLineageCheck = skipSegmentLineageCheck;
|
this.skipSegmentLineageCheck = skipSegmentLineageCheck;
|
||||||
this.shardSpecFactory = shardSpecFactory == null ? NumberedShardSpecFactory.instance() : shardSpecFactory;
|
this.partialShardSpec = partialShardSpec == null ? NumberedPartialShardSpec.instance() : partialShardSpec;
|
||||||
this.lockGranularity = lockGranularity == null ? LockGranularity.TIME_CHUNK : lockGranularity;
|
this.lockGranularity = lockGranularity == null ? LockGranularity.TIME_CHUNK : lockGranularity;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -148,10 +150,10 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
return skipSegmentLineageCheck;
|
return skipSegmentLineageCheck;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty("shardSpecFactory")
|
||||||
public ShardSpecFactory getShardSpecFactory()
|
public PartialShardSpec getPartialShardSpec()
|
||||||
{
|
{
|
||||||
return shardSpecFactory;
|
return partialShardSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -293,7 +295,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
task.getGroupId(),
|
task.getGroupId(),
|
||||||
dataSource,
|
dataSource,
|
||||||
tryInterval,
|
tryInterval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
task.getPriority(),
|
task.getPriority(),
|
||||||
sequenceName,
|
sequenceName,
|
||||||
previousSegmentId,
|
previousSegmentId,
|
||||||
|
@ -355,7 +357,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
|
||||||
", sequenceName='" + sequenceName + '\'' +
|
", sequenceName='" + sequenceName + '\'' +
|
||||||
", previousSegmentId='" + previousSegmentId + '\'' +
|
", previousSegmentId='" + previousSegmentId + '\'' +
|
||||||
", skipSegmentLineageCheck=" + skipSegmentLineageCheck +
|
", skipSegmentLineageCheck=" + skipSegmentLineageCheck +
|
||||||
", shardSpecFactory=" + shardSpecFactory +
|
", partialShardSpec=" + partialShardSpec +
|
||||||
", lockGranularity=" + lockGranularity +
|
", lockGranularity=" + lockGranularity +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,8 +27,8 @@ import com.google.common.collect.ImmutableSet;
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
import org.apache.druid.indexing.common.LockGranularity;
|
||||||
import org.apache.druid.indexing.common.TaskLock;
|
import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
import org.apache.druid.indexing.common.task.IndexTaskUtils;
|
||||||
import org.apache.druid.indexing.common.task.SegmentLockHelper;
|
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
|
import org.apache.druid.indexing.common.task.TaskLockHelper;
|
||||||
import org.apache.druid.indexing.overlord.CriticalAction;
|
import org.apache.druid.indexing.overlord.CriticalAction;
|
||||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||||
import org.apache.druid.indexing.overlord.SegmentPublishResult;
|
import org.apache.druid.indexing.overlord.SegmentPublishResult;
|
||||||
|
@ -238,8 +238,8 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
|
||||||
final Map<Interval, List<DataSegment>> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten);
|
final Map<Interval, List<DataSegment>> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten);
|
||||||
final Map<Interval, List<DataSegment>> newSegmentsMap = groupSegmentsByIntervalAndSort(segments);
|
final Map<Interval, List<DataSegment>> newSegmentsMap = groupSegmentsByIntervalAndSort(segments);
|
||||||
|
|
||||||
oldSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
|
oldSegmentsMap.values().forEach(TaskLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
|
||||||
newSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
|
newSegmentsMap.values().forEach(TaskLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
|
||||||
|
|
||||||
oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> {
|
oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> {
|
||||||
final List<DataSegment> newSegmentsPerInterval = Preconditions.checkNotNull(
|
final List<DataSegment> newSegmentsPerInterval = Preconditions.checkNotNull(
|
||||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.druid.indexing.common.task.Task;
|
||||||
@JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class),
|
@JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class),
|
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class),
|
@JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class),
|
@JsonSubTypes.Type(name = SegmentAllocateAction.TYPE, value = SegmentAllocateAction.class),
|
||||||
@JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class),
|
@JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class),
|
||||||
@JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class),
|
@JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class),
|
||||||
@JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class)
|
@JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class)
|
||||||
|
|
|
@ -24,8 +24,6 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import org.apache.druid.data.input.FirehoseFactory;
|
import org.apache.druid.data.input.FirehoseFactory;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
import org.apache.druid.indexing.common.LockGranularity;
|
||||||
import org.apache.druid.indexing.common.TaskLock;
|
import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskLockType;
|
import org.apache.druid.indexing.common.TaskLockType;
|
||||||
|
@ -40,7 +38,6 @@ import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
|
||||||
import org.apache.druid.indexing.firehose.WindowedSegmentId;
|
import org.apache.druid.indexing.firehose.WindowedSegmentId;
|
||||||
import org.apache.druid.indexing.overlord.Segments;
|
import org.apache.druid.indexing.overlord.Segments;
|
||||||
import org.apache.druid.java.util.common.JodaUtils;
|
import org.apache.druid.java.util.common.JodaUtils;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.java.util.common.granularity.GranularityType;
|
import org.apache.druid.java.util.common.granularity.GranularityType;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
@ -48,24 +45,20 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.Partitions;
|
import org.apache.druid.timeline.Partitions;
|
||||||
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
|
|
||||||
import javax.annotation.Nonnull;
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import javax.annotation.concurrent.GuardedBy;
|
import javax.annotation.concurrent.GuardedBy;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.SortedSet;
|
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ -78,24 +71,17 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(AbstractBatchIndexTask.class);
|
private static final Logger log = new Logger(AbstractBatchIndexTask.class);
|
||||||
|
|
||||||
private final SegmentLockHelper segmentLockHelper;
|
|
||||||
|
|
||||||
@GuardedBy("this")
|
@GuardedBy("this")
|
||||||
private final TaskResourceCleaner resourceCloserOnAbnormalExit = new TaskResourceCleaner();
|
private final TaskResourceCleaner resourceCloserOnAbnormalExit = new TaskResourceCleaner();
|
||||||
|
|
||||||
/**
|
|
||||||
* State to indicate that this task will use segmentLock or timeChunkLock.
|
|
||||||
* This is automatically set when {@link #determineLockGranularityandTryLock} is called.
|
|
||||||
*/
|
|
||||||
private boolean useSegmentLock;
|
|
||||||
|
|
||||||
@GuardedBy("this")
|
@GuardedBy("this")
|
||||||
private boolean stopped = false;
|
private boolean stopped = false;
|
||||||
|
|
||||||
|
private TaskLockHelper taskLockHelper;
|
||||||
|
|
||||||
protected AbstractBatchIndexTask(String id, String dataSource, Map<String, Object> context)
|
protected AbstractBatchIndexTask(String id, String dataSource, Map<String, Object> context)
|
||||||
{
|
{
|
||||||
super(id, dataSource, context);
|
super(id, dataSource, context);
|
||||||
segmentLockHelper = new SegmentLockHelper();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected AbstractBatchIndexTask(
|
protected AbstractBatchIndexTask(
|
||||||
|
@ -107,7 +93,6 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(id, groupId, taskResource, dataSource, context);
|
super(id, groupId, taskResource, dataSource, context);
|
||||||
segmentLockHelper = new SegmentLockHelper();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -194,14 +179,9 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
|
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isUseSegmentLock()
|
public TaskLockHelper getTaskLockHelper()
|
||||||
{
|
{
|
||||||
return useSegmentLock;
|
return Preconditions.checkNotNull(taskLockHelper, "taskLockHelper is not initialized yet");
|
||||||
}
|
|
||||||
|
|
||||||
public SegmentLockHelper getSegmentLockHelper()
|
|
||||||
{
|
|
||||||
return segmentLockHelper;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -229,7 +209,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
// Respect task context value most.
|
// Respect task context value most.
|
||||||
if (forceTimeChunkLock) {
|
if (forceTimeChunkLock) {
|
||||||
log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
|
log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
|
||||||
useSegmentLock = false;
|
taskLockHelper = new TaskLockHelper(false);
|
||||||
if (!intervals.isEmpty()) {
|
if (!intervals.isEmpty()) {
|
||||||
return tryTimeChunkLock(client, intervals);
|
return tryTimeChunkLock(client, intervals);
|
||||||
} else {
|
} else {
|
||||||
|
@ -238,7 +218,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
} else {
|
} else {
|
||||||
if (!intervals.isEmpty()) {
|
if (!intervals.isEmpty()) {
|
||||||
final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals);
|
final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals);
|
||||||
useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT;
|
taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT);
|
||||||
return tryLockWithDetermineResult(client, result);
|
return tryLockWithDetermineResult(client, result);
|
||||||
} else {
|
} else {
|
||||||
return true;
|
return true;
|
||||||
|
@ -255,14 +235,14 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
);
|
);
|
||||||
if (forceTimeChunkLock) {
|
if (forceTimeChunkLock) {
|
||||||
log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
|
log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
|
||||||
useSegmentLock = false;
|
taskLockHelper = new TaskLockHelper(false);
|
||||||
return tryTimeChunkLock(
|
return tryTimeChunkLock(
|
||||||
client,
|
client,
|
||||||
new ArrayList<>(segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()))
|
new ArrayList<>(segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()))
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
final LockGranularityDetermineResult result = determineSegmentGranularity(segments);
|
final LockGranularityDetermineResult result = determineSegmentGranularity(segments);
|
||||||
useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT;
|
taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT);
|
||||||
return tryLockWithDetermineResult(client, result);
|
return tryLockWithDetermineResult(client, result);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -298,7 +278,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
if (result.lockGranularity == LockGranularity.TIME_CHUNK) {
|
if (result.lockGranularity == LockGranularity.TIME_CHUNK) {
|
||||||
return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals"));
|
return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals"));
|
||||||
} else {
|
} else {
|
||||||
return segmentLockHelper.verifyAndLockExistingSegments(
|
return taskLockHelper.verifyAndLockExistingSegments(
|
||||||
client,
|
client,
|
||||||
Preconditions.checkNotNull(result.segments, "segments")
|
Preconditions.checkNotNull(result.segments, "segments")
|
||||||
);
|
);
|
||||||
|
@ -377,7 +357,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
/**
|
/**
|
||||||
* We currently don't support appending perfectly rolled up segments. This might be supported in the future if there
|
* We currently don't support appending perfectly rolled up segments. This might be supported in the future if there
|
||||||
* is a good use case. If we want to support appending perfectly rolled up segments, we need to fix some other places
|
* is a good use case. If we want to support appending perfectly rolled up segments, we need to fix some other places
|
||||||
* first. For example, {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec#getLookup} assumes that
|
* first. For example, {@link HashBasedNumberedShardSpec#getLookup} assumes that
|
||||||
* the start partition ID of the set of perfectly rolled up segments is 0. Instead it might need to store an ordinal
|
* the start partition ID of the set of perfectly rolled up segments is 0. Instead it might need to store an ordinal
|
||||||
* in addition to the partition ID which represents the ordinal in the perfectly rolled up segment set.
|
* in addition to the partition ID which represents the ordinal in the perfectly rolled up segment set.
|
||||||
*/
|
*/
|
||||||
|
@ -390,14 +370,6 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
return tuningConfig.isForceGuaranteedRollup();
|
return tuningConfig.isForceGuaranteedRollup();
|
||||||
}
|
}
|
||||||
|
|
||||||
static Pair<ShardSpecFactory, Integer> createShardSpecFactoryForGuaranteedRollup(
|
|
||||||
int numShards,
|
|
||||||
@Nullable List<String> partitionDimensions
|
|
||||||
)
|
|
||||||
{
|
|
||||||
return Pair.of(new HashBasedNumberedShardSpecFactory(partitionDimensions, numShards), numShards);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
static Granularity findGranularityFromSegments(List<DataSegment> segments)
|
static Granularity findGranularityFromSegments(List<DataSegment> segments)
|
||||||
{
|
{
|
||||||
|
@ -415,45 +387,6 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates shard specs based on the given configurations. The return value is a map between intervals created
|
|
||||||
* based on the segment granularity and the shard specs to be created.
|
|
||||||
* Note that the shard specs to be created is a pair of {@link ShardSpecFactory} and number of segments per interval
|
|
||||||
* and filled only when {@link #isGuaranteedRollup} = true. Otherwise, the return value contains only the set of
|
|
||||||
* intervals generated based on the segment granularity.
|
|
||||||
*/
|
|
||||||
protected static Map<Interval, Pair<ShardSpecFactory, Integer>> createShardSpecWithoutInputScan(
|
|
||||||
GranularitySpec granularitySpec,
|
|
||||||
IndexIOConfig ioConfig,
|
|
||||||
IndexTuningConfig tuningConfig,
|
|
||||||
@Nonnull PartitionsSpec nonNullPartitionsSpec
|
|
||||||
)
|
|
||||||
{
|
|
||||||
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = new HashMap<>();
|
|
||||||
final SortedSet<Interval> intervals = granularitySpec.bucketIntervals().get();
|
|
||||||
|
|
||||||
if (isGuaranteedRollup(ioConfig, tuningConfig)) {
|
|
||||||
// SingleDimensionPartitionsSpec or more partitionsSpec types will be supported in the future.
|
|
||||||
assert nonNullPartitionsSpec instanceof HashedPartitionsSpec;
|
|
||||||
// Overwrite mode, guaranteed rollup: shardSpecs must be known in advance.
|
|
||||||
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) nonNullPartitionsSpec;
|
|
||||||
final int numShards = partitionsSpec.getNumShards() == null ? 1 : partitionsSpec.getNumShards();
|
|
||||||
|
|
||||||
for (Interval interval : intervals) {
|
|
||||||
allocateSpec.put(
|
|
||||||
interval,
|
|
||||||
createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions())
|
|
||||||
);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
for (Interval interval : intervals) {
|
|
||||||
allocateSpec.put(interval, null);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return allocateSpec;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock
|
* If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock
|
||||||
* from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be
|
* from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be
|
||||||
|
|
|
@ -90,7 +90,7 @@ import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory;
|
||||||
import org.apache.druid.segment.realtime.plumber.Committers;
|
import org.apache.druid.segment.realtime.plumber.Committers;
|
||||||
import org.apache.druid.server.security.Action;
|
import org.apache.druid.server.security.Action;
|
||||||
import org.apache.druid.server.security.AuthorizerMapper;
|
import org.apache.druid.server.security.AuthorizerMapper;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
import org.apache.druid.utils.CircularBuffer;
|
import org.apache.druid.utils.CircularBuffer;
|
||||||
|
|
||||||
import javax.servlet.http.HttpServletRequest;
|
import javax.servlet.http.HttpServletRequest;
|
||||||
|
@ -799,7 +799,7 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
|
||||||
sequenceName,
|
sequenceName,
|
||||||
previousSegmentId,
|
previousSegmentId,
|
||||||
skipSegmentLineageCheck,
|
skipSegmentLineageCheck,
|
||||||
NumberedShardSpecFactory.instance(),
|
NumberedPartialShardSpec.instance(),
|
||||||
LockGranularity.TIME_CHUNK
|
LockGranularity.TIME_CHUNK
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
|
|
|
@ -19,17 +19,21 @@
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.task;
|
package org.apache.druid.indexing.common.task;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.druid.data.input.InputRow;
|
import org.apache.druid.data.input.InputRow;
|
||||||
import org.apache.druid.indexing.common.TaskLock;
|
import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.LockListAction;
|
import org.apache.druid.indexing.common.actions.LockListAction;
|
||||||
import org.apache.druid.indexing.common.actions.SurrogateAction;
|
import org.apache.druid.indexing.common.actions.SurrogateAction;
|
||||||
|
import org.apache.druid.indexing.common.actions.TaskAction;
|
||||||
import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs;
|
import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpec;
|
import org.apache.druid.timeline.partition.ShardSpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
@ -41,11 +45,8 @@ import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Allocates all necessary segments locally at the beginning and reuses them.
|
* Allocates all necessary segments locally at the beginning and reuses them.
|
||||||
*
|
|
||||||
* @see HashPartitionCachingLocalSegmentAllocator
|
|
||||||
* @see RangePartitionCachingLocalSegmentAllocator
|
|
||||||
*/
|
*/
|
||||||
class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
|
public class CachingLocalSegmentAllocator implements CachingSegmentAllocator
|
||||||
{
|
{
|
||||||
private final String taskId;
|
private final String taskId;
|
||||||
private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
|
private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
|
||||||
|
@ -59,22 +60,34 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
|
||||||
*
|
*
|
||||||
* @return Information for segment preallocation
|
* @return Information for segment preallocation
|
||||||
*/
|
*/
|
||||||
Map<Interval, List<SegmentIdWithShardSpec>> create(Function<Interval, String> versionFinder);
|
Map<Interval, List<SegmentIdWithShardSpec>> create(
|
||||||
|
TaskToolbox toolbox,
|
||||||
|
String dataSource,
|
||||||
|
Function<Interval, String> versionFinder
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
CachingLocalSegmentAllocatorHelper(
|
CachingLocalSegmentAllocator(
|
||||||
TaskToolbox toolbox,
|
TaskToolbox toolbox,
|
||||||
|
String dataSource,
|
||||||
String taskId,
|
String taskId,
|
||||||
String supervisorTaskId,
|
@Nullable SupervisorTaskAccess supervisorTaskAccess,
|
||||||
IntervalToSegmentIdsCreator intervalToSegmentIdsCreator
|
IntervalToSegmentIdsCreator intervalToSegmentIdsCreator
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
this.taskId = taskId;
|
this.taskId = taskId;
|
||||||
this.sequenceNameToSegmentId = new HashMap<>();
|
this.sequenceNameToSegmentId = new HashMap<>();
|
||||||
|
|
||||||
|
final TaskAction<List<TaskLock>> action;
|
||||||
|
if (supervisorTaskAccess == null) {
|
||||||
|
action = new LockListAction();
|
||||||
|
} else {
|
||||||
|
action = new SurrogateAction<>(supervisorTaskAccess.getSupervisorTaskId(), new LockListAction());
|
||||||
|
}
|
||||||
|
|
||||||
final Map<Interval, String> intervalToVersion =
|
final Map<Interval, String> intervalToVersion =
|
||||||
toolbox.getTaskActionClient()
|
toolbox.getTaskActionClient()
|
||||||
.submit(new SurrogateAction<>(supervisorTaskId, new LockListAction()))
|
.submit(action)
|
||||||
.stream()
|
.stream()
|
||||||
.collect(Collectors.toMap(
|
.collect(Collectors.toMap(
|
||||||
TaskLock::getInterval,
|
TaskLock::getInterval,
|
||||||
|
@ -82,7 +95,11 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
|
||||||
));
|
));
|
||||||
Function<Interval, String> versionFinder = interval -> findVersion(intervalToVersion, interval);
|
Function<Interval, String> versionFinder = interval -> findVersion(intervalToVersion, interval);
|
||||||
|
|
||||||
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToIds = intervalToSegmentIdsCreator.create(versionFinder);
|
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToIds = intervalToSegmentIdsCreator.create(
|
||||||
|
toolbox,
|
||||||
|
dataSource,
|
||||||
|
versionFinder
|
||||||
|
);
|
||||||
final Map<Interval, List<ShardSpec>> shardSpecMap = new HashMap<>();
|
final Map<Interval, List<ShardSpec>> shardSpecMap = new HashMap<>();
|
||||||
|
|
||||||
for (Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) {
|
for (Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) {
|
||||||
|
@ -115,14 +132,11 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
|
||||||
boolean skipSegmentLineageCheck
|
boolean skipSegmentLineageCheck
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return sequenceNameToSegmentId.get(sequenceName);
|
return Preconditions.checkNotNull(
|
||||||
}
|
sequenceNameToSegmentId.get(sequenceName),
|
||||||
|
"Missing segmentId for the sequence[%s]",
|
||||||
@Override
|
sequenceName
|
||||||
public String getSequenceName(Interval interval, InputRow inputRow)
|
);
|
||||||
{
|
|
||||||
// Sequence name is based solely on the shardSpec, and there will only be one segment per sequence.
|
|
||||||
return getSequenceName(interval, shardSpecs.getShardSpec(interval, inputRow));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -136,4 +150,10 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
|
||||||
// and it's faster to add strings together than it is to use String#format
|
// and it's faster to add strings together than it is to use String#format
|
||||||
return taskId + "_" + interval + "_" + shardSpec.getPartitionNum();
|
return taskId + "_" + interval + "_" + shardSpec.getPartitionNum();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ShardSpecs getShardSpecs()
|
||||||
|
{
|
||||||
|
return shardSpecs;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -0,0 +1,39 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.druid.indexing.common.task.IndexTask.ShardSpecs;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* SegmentAllocator that allocates all necessary segments upfront. This allocator should be used for the hash or range
|
||||||
|
* secondary partitioning.
|
||||||
|
*
|
||||||
|
* In the hash or range secondary partitioning, the information about all partition buckets should be known before
|
||||||
|
* the task starts to allocate segments. For example, for the hash partitioning, the task should know how many hash
|
||||||
|
* buckets it will create, what is the hash value allocated for each bucket, etc. Similar for the range partitioning.
|
||||||
|
*/
|
||||||
|
public interface CachingSegmentAllocator extends SegmentAllocator
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* Returns the {@link org.apache.druid.timeline.partition.ShardSpec}s of all segments allocated upfront.
|
||||||
|
*/
|
||||||
|
ShardSpecs getShardSpecs();
|
||||||
|
}
|
|
@ -1,116 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing,
|
|
||||||
* software distributed under the License is distributed on an
|
|
||||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
||||||
* KIND, either express or implied. See the License for the
|
|
||||||
* specific language governing permissions and limitations
|
|
||||||
* under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.task;
|
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
|
||||||
import com.google.common.collect.Maps;
|
|
||||||
import org.apache.druid.data.input.InputRow;
|
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
|
||||||
import org.apache.druid.java.util.common.Pair;
|
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.joda.time.Interval;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.function.Function;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
import java.util.stream.IntStream;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Allocates all necessary hash-partitioned segments locally at the beginning and reuses them.
|
|
||||||
*
|
|
||||||
* @see CachingLocalSegmentAllocatorHelper
|
|
||||||
*/
|
|
||||||
public class HashPartitionCachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
|
|
||||||
{
|
|
||||||
private final TaskToolbox toolbox;
|
|
||||||
private final String dataSource;
|
|
||||||
private final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec;
|
|
||||||
private final IndexTaskSegmentAllocator delegate;
|
|
||||||
|
|
||||||
public HashPartitionCachingLocalSegmentAllocator(
|
|
||||||
TaskToolbox toolbox,
|
|
||||||
String taskId,
|
|
||||||
String supervisorTaskId,
|
|
||||||
String dataSource,
|
|
||||||
Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
this.toolbox = toolbox;
|
|
||||||
this.dataSource = dataSource;
|
|
||||||
this.allocateSpec = allocateSpec;
|
|
||||||
|
|
||||||
this.delegate = new CachingLocalSegmentAllocatorHelper(
|
|
||||||
toolbox,
|
|
||||||
taskId,
|
|
||||||
supervisorTaskId,
|
|
||||||
this::getIntervalToSegmentIds
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
private Map<Interval, List<SegmentIdWithShardSpec>> getIntervalToSegmentIds(Function<Interval, String> versionFinder)
|
|
||||||
{
|
|
||||||
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToSegmentIds =
|
|
||||||
Maps.newHashMapWithExpectedSize(allocateSpec.size());
|
|
||||||
|
|
||||||
for (Entry<Interval, Pair<ShardSpecFactory, Integer>> entry : allocateSpec.entrySet()) {
|
|
||||||
final Interval interval = entry.getKey();
|
|
||||||
final ShardSpecFactory shardSpecFactory = entry.getValue().lhs;
|
|
||||||
final int numSegmentsToAllocate = Preconditions.checkNotNull(
|
|
||||||
entry.getValue().rhs,
|
|
||||||
"numSegmentsToAllocate for interval[%s]",
|
|
||||||
interval
|
|
||||||
);
|
|
||||||
|
|
||||||
intervalToSegmentIds.put(
|
|
||||||
interval,
|
|
||||||
IntStream.range(0, numSegmentsToAllocate)
|
|
||||||
.mapToObj(i -> new SegmentIdWithShardSpec(
|
|
||||||
dataSource,
|
|
||||||
interval,
|
|
||||||
versionFinder.apply(interval),
|
|
||||||
shardSpecFactory.create(toolbox.getJsonMapper(), i)
|
|
||||||
))
|
|
||||||
.collect(Collectors.toList())
|
|
||||||
);
|
|
||||||
}
|
|
||||||
return intervalToSegmentIds;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getSequenceName(Interval interval, InputRow inputRow)
|
|
||||||
{
|
|
||||||
return delegate.getSequenceName(interval, inputRow);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public SegmentIdWithShardSpec allocate(
|
|
||||||
InputRow row,
|
|
||||||
String sequenceName,
|
|
||||||
String previousSegmentId,
|
|
||||||
boolean skipSegmentLineageCheck
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
return delegate.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -53,9 +53,9 @@ import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import org.apache.druid.indexer.partitions.SecondaryPartitionType;
|
||||||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
|
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
|
||||||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
|
||||||
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
|
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
|
||||||
import org.apache.druid.indexing.common.TaskReport;
|
import org.apache.druid.indexing.common.TaskReport;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
@ -64,14 +64,19 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
import org.apache.druid.indexing.common.stats.RowIngestionMeters;
|
import org.apache.druid.indexing.common.stats.RowIngestionMeters;
|
||||||
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
||||||
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
|
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder;
|
import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.LinearPartitionAnalysis;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.PartitionAnalysis;
|
||||||
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
|
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.JodaUtils;
|
import org.apache.druid.java.util.common.JodaUtils;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
import org.apache.druid.java.util.common.UOE;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.java.util.common.guava.Comparators;
|
import org.apache.druid.java.util.common.guava.Comparators;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
|
@ -94,6 +99,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
||||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||||
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
|
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
|
||||||
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
||||||
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
|
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
|
||||||
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
||||||
|
@ -105,12 +111,12 @@ import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpec;
|
import org.apache.druid.timeline.partition.ShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.apache.druid.utils.CircularBuffer;
|
import org.apache.druid.utils.CircularBuffer;
|
||||||
import org.codehaus.plexus.util.FileUtils;
|
import org.codehaus.plexus.util.FileUtils;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
|
|
||||||
|
import javax.annotation.Nonnull;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import javax.servlet.http.HttpServletRequest;
|
import javax.servlet.http.HttpServletRequest;
|
||||||
import javax.ws.rs.GET;
|
import javax.ws.rs.GET;
|
||||||
|
@ -130,6 +136,7 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.SortedSet;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -264,6 +271,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
@Override
|
@Override
|
||||||
public boolean isReady(TaskActionClient taskActionClient) throws Exception
|
public boolean isReady(TaskActionClient taskActionClient) throws Exception
|
||||||
{
|
{
|
||||||
|
final IndexTuningConfig tuningConfig = getIngestionSchema().getTuningConfig();
|
||||||
|
if (tuningConfig != null && tuningConfig.getPartitionsSpec() != null) {
|
||||||
|
if (tuningConfig.getPartitionsSpec().getType() != SecondaryPartitionType.LINEAR
|
||||||
|
&& tuningConfig.getPartitionsSpec().getType() != SecondaryPartitionType.HASH) {
|
||||||
|
throw new UOE("partitionsSpec[%s] is not supported", tuningConfig.getPartitionsSpec().getClass().getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.dataSchema.getGranularitySpec());
|
return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.dataSchema.getGranularitySpec());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -487,13 +501,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
// Initialize maxRowsPerSegment and maxTotalRows lazily
|
// Initialize maxRowsPerSegment and maxTotalRows lazily
|
||||||
final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig;
|
final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig;
|
||||||
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
|
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
|
||||||
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = determineShardSpecs(
|
final PartitionAnalysis partitionAnalysis = determineShardSpecs(
|
||||||
toolbox,
|
toolbox,
|
||||||
inputSource,
|
inputSource,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
partitionsSpec
|
partitionsSpec
|
||||||
);
|
);
|
||||||
final List<Interval> allocateIntervals = new ArrayList<>(allocateSpec.keySet());
|
final List<Interval> allocateIntervals = new ArrayList<>(partitionAnalysis.getAllIntervalsToIndex());
|
||||||
final DataSchema dataSchema;
|
final DataSchema dataSchema;
|
||||||
if (determineIntervals) {
|
if (determineIntervals) {
|
||||||
if (!determineLockGranularityandTryLock(toolbox.getTaskActionClient(), allocateIntervals)) {
|
if (!determineLockGranularityandTryLock(toolbox.getTaskActionClient(), allocateIntervals)) {
|
||||||
|
@ -508,14 +522,14 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
} else {
|
} else {
|
||||||
dataSchema = ingestionSchema.getDataSchema();
|
dataSchema = ingestionSchema.getDataSchema();
|
||||||
}
|
}
|
||||||
|
|
||||||
ingestionState = IngestionState.BUILD_SEGMENTS;
|
ingestionState = IngestionState.BUILD_SEGMENTS;
|
||||||
return generateAndPublishSegments(
|
return generateAndPublishSegments(
|
||||||
toolbox,
|
toolbox,
|
||||||
dataSchema,
|
dataSchema,
|
||||||
allocateSpec,
|
|
||||||
inputSource,
|
inputSource,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
partitionsSpec
|
partitionAnalysis
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
@ -598,16 +612,14 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
*
|
*
|
||||||
* @return a map indicating how many shardSpecs need to be created per interval.
|
* @return a map indicating how many shardSpecs need to be created per interval.
|
||||||
*/
|
*/
|
||||||
private Map<Interval, Pair<ShardSpecFactory, Integer>> determineShardSpecs(
|
private PartitionAnalysis determineShardSpecs(
|
||||||
final TaskToolbox toolbox,
|
final TaskToolbox toolbox,
|
||||||
final InputSource inputSource,
|
final InputSource inputSource,
|
||||||
final File tmpDir,
|
final File tmpDir,
|
||||||
final PartitionsSpec nonNullPartitionsSpec
|
@Nonnull final PartitionsSpec partitionsSpec
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
final ObjectMapper jsonMapper = toolbox.getJsonMapper();
|
final ObjectMapper jsonMapper = toolbox.getJsonMapper();
|
||||||
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
|
||||||
final IndexIOConfig ioConfig = ingestionSchema.getIOConfig();
|
|
||||||
|
|
||||||
final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
|
final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
|
||||||
|
|
||||||
|
@ -615,42 +627,59 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent();
|
final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent();
|
||||||
|
|
||||||
// Must determine partitions if rollup is guaranteed and the user didn't provide a specific value.
|
// Must determine partitions if rollup is guaranteed and the user didn't provide a specific value.
|
||||||
final boolean determineNumPartitions = nonNullPartitionsSpec.needsDeterminePartitions(false);
|
final boolean determineNumPartitions = partitionsSpec.needsDeterminePartitions(false);
|
||||||
|
|
||||||
// if we were given number of shards per interval and the intervals, we don't need to scan the data
|
// if we were given number of shards per interval and the intervals, we don't need to scan the data
|
||||||
if (!determineNumPartitions && !determineIntervals) {
|
if (!determineNumPartitions && !determineIntervals) {
|
||||||
log.info("Skipping determine partition scan");
|
log.info("Skipping determine partition scan");
|
||||||
return createShardSpecWithoutInputScan(
|
if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
|
||||||
|
return PartialHashSegmentGenerateTask.createHashPartitionAnalysisFromPartitionsSpec(
|
||||||
granularitySpec,
|
granularitySpec,
|
||||||
ioConfig,
|
(HashedPartitionsSpec) partitionsSpec
|
||||||
tuningConfig,
|
|
||||||
nonNullPartitionsSpec
|
|
||||||
);
|
);
|
||||||
|
} else if (partitionsSpec.getType() == SecondaryPartitionType.LINEAR) {
|
||||||
|
return createLinearPartitionAnalysis(granularitySpec, (DynamicPartitionsSpec) partitionsSpec);
|
||||||
|
} else {
|
||||||
|
throw new UOE("%s", partitionsSpec.getClass().getName());
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
// determine intervals containing data and prime HLL collectors
|
// determine intervals containing data and prime HLL collectors
|
||||||
|
log.info("Determining intervals and shardSpecs");
|
||||||
return createShardSpecsFromInput(
|
return createShardSpecsFromInput(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
ingestionSchema,
|
ingestionSchema,
|
||||||
inputSource,
|
inputSource,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
granularitySpec,
|
granularitySpec,
|
||||||
nonNullPartitionsSpec,
|
partitionsSpec,
|
||||||
determineIntervals
|
determineIntervals
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<Interval, Pair<ShardSpecFactory, Integer>> createShardSpecsFromInput(
|
private static LinearPartitionAnalysis createLinearPartitionAnalysis(
|
||||||
|
GranularitySpec granularitySpec,
|
||||||
|
@Nonnull DynamicPartitionsSpec partitionsSpec
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final SortedSet<Interval> intervals = granularitySpec.bucketIntervals().get();
|
||||||
|
final int numBucketsPerInterval = 1;
|
||||||
|
final LinearPartitionAnalysis partitionAnalysis = new LinearPartitionAnalysis(partitionsSpec);
|
||||||
|
intervals.forEach(interval -> partitionAnalysis.updateBucket(interval, numBucketsPerInterval));
|
||||||
|
return partitionAnalysis;
|
||||||
|
}
|
||||||
|
|
||||||
|
private PartitionAnalysis createShardSpecsFromInput(
|
||||||
ObjectMapper jsonMapper,
|
ObjectMapper jsonMapper,
|
||||||
IndexIngestionSpec ingestionSchema,
|
IndexIngestionSpec ingestionSchema,
|
||||||
InputSource inputSource,
|
InputSource inputSource,
|
||||||
File tmpDir,
|
File tmpDir,
|
||||||
GranularitySpec granularitySpec,
|
GranularitySpec granularitySpec,
|
||||||
PartitionsSpec nonNullPartitionsSpec,
|
@Nonnull PartitionsSpec partitionsSpec,
|
||||||
boolean determineIntervals
|
boolean determineIntervals
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
log.info("Determining intervals and shardSpecs");
|
assert partitionsSpec.getType() != SecondaryPartitionType.RANGE;
|
||||||
long determineShardSpecsStartMillis = System.currentTimeMillis();
|
long determineShardSpecsStartMillis = System.currentTimeMillis();
|
||||||
|
|
||||||
final Map<Interval, Optional<HyperLogLogCollector>> hllCollectors = collectIntervalsAndShardSpecs(
|
final Map<Interval, Optional<HyperLogLogCollector>> hllCollectors = collectIntervalsAndShardSpecs(
|
||||||
|
@ -659,50 +688,49 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
inputSource,
|
inputSource,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
granularitySpec,
|
granularitySpec,
|
||||||
nonNullPartitionsSpec,
|
partitionsSpec,
|
||||||
determineIntervals
|
determineIntervals
|
||||||
);
|
);
|
||||||
|
final PartitionAnalysis<Integer, ?> partitionAnalysis;
|
||||||
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpecs = new HashMap<>();
|
if (partitionsSpec.getType() == SecondaryPartitionType.LINEAR) {
|
||||||
|
partitionAnalysis = new LinearPartitionAnalysis((DynamicPartitionsSpec) partitionsSpec);
|
||||||
|
} else if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
|
||||||
|
partitionAnalysis = new HashPartitionAnalysis((HashedPartitionsSpec) partitionsSpec);
|
||||||
|
} else {
|
||||||
|
throw new UOE("%s", partitionsSpec.getClass().getName());
|
||||||
|
}
|
||||||
for (final Map.Entry<Interval, Optional<HyperLogLogCollector>> entry : hllCollectors.entrySet()) {
|
for (final Map.Entry<Interval, Optional<HyperLogLogCollector>> entry : hllCollectors.entrySet()) {
|
||||||
final Interval interval = entry.getKey();
|
final Interval interval = entry.getKey();
|
||||||
|
final int numBucketsPerInterval;
|
||||||
if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) {
|
if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
|
||||||
assert nonNullPartitionsSpec instanceof HashedPartitionsSpec;
|
final HashedPartitionsSpec hashedPartitionsSpec = (HashedPartitionsSpec) partitionsSpec;
|
||||||
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) nonNullPartitionsSpec;
|
|
||||||
|
|
||||||
final HyperLogLogCollector collector = entry.getValue().orNull();
|
final HyperLogLogCollector collector = entry.getValue().orNull();
|
||||||
|
|
||||||
final int numShards;
|
|
||||||
if (partitionsSpec.needsDeterminePartitions(false)) {
|
if (partitionsSpec.needsDeterminePartitions(false)) {
|
||||||
final long numRows = Preconditions.checkNotNull(collector, "HLL collector").estimateCardinalityRound();
|
final long numRows = Preconditions.checkNotNull(collector, "HLL collector").estimateCardinalityRound();
|
||||||
final int nonNullMaxRowsPerSegment = partitionsSpec.getMaxRowsPerSegment() == null
|
final int nonNullMaxRowsPerSegment = partitionsSpec.getMaxRowsPerSegment() == null
|
||||||
? PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT
|
? PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT
|
||||||
: partitionsSpec.getMaxRowsPerSegment();
|
: partitionsSpec.getMaxRowsPerSegment();
|
||||||
numShards = (int) Math.ceil((double) numRows / nonNullMaxRowsPerSegment);
|
numBucketsPerInterval = (int) Math.ceil((double) numRows / nonNullMaxRowsPerSegment);
|
||||||
log.info(
|
log.info(
|
||||||
"Estimated [%,d] rows of data for interval [%s], creating [%,d] shards",
|
"Estimated [%,d] rows of data for interval [%s], creating [%,d] shards",
|
||||||
numRows,
|
numRows,
|
||||||
interval,
|
interval,
|
||||||
numShards
|
numBucketsPerInterval
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
numShards = partitionsSpec.getNumShards() == null ? 1 : partitionsSpec.getNumShards();
|
numBucketsPerInterval = hashedPartitionsSpec.getNumShards() == null ? 1 : hashedPartitionsSpec.getNumShards();
|
||||||
log.info("Creating [%,d] shards for interval [%s]", numShards, interval);
|
log.info("Creating [%,d] buckets for interval [%s]", numBucketsPerInterval, interval);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
numBucketsPerInterval = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Overwrite mode, guaranteed rollup: shardSpecs must be known in advance.
|
partitionAnalysis.updateBucket(interval, numBucketsPerInterval);
|
||||||
allocateSpecs.put(
|
|
||||||
interval,
|
|
||||||
createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions())
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
allocateSpecs.put(interval, null);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis);
|
log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis);
|
||||||
|
|
||||||
return allocateSpecs;
|
return partitionAnalysis;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<Interval, Optional<HyperLogLogCollector>> collectIntervalsAndShardSpecs(
|
private Map<Interval, Optional<HyperLogLogCollector>> collectIntervalsAndShardSpecs(
|
||||||
|
@ -711,7 +739,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
InputSource inputSource,
|
InputSource inputSource,
|
||||||
File tmpDir,
|
File tmpDir,
|
||||||
GranularitySpec granularitySpec,
|
GranularitySpec granularitySpec,
|
||||||
PartitionsSpec nonNullPartitionsSpec,
|
@Nonnull PartitionsSpec partitionsSpec,
|
||||||
boolean determineIntervals
|
boolean determineIntervals
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -765,7 +793,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
interval = optInterval.get();
|
interval = optInterval.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (nonNullPartitionsSpec.needsDeterminePartitions(false)) {
|
if (partitionsSpec.needsDeterminePartitions(false)) {
|
||||||
hllCollectors.computeIfAbsent(interval, intv -> Optional.of(HyperLogLogCollector.makeLatestCollector()));
|
hllCollectors.computeIfAbsent(interval, intv -> Optional.of(HyperLogLogCollector.makeLatestCollector()));
|
||||||
|
|
||||||
List<Object> groupKey = Rows.toGroupKey(
|
List<Object> groupKey = Rows.toGroupKey(
|
||||||
|
@ -809,32 +837,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
return hllCollectors;
|
return hllCollectors;
|
||||||
}
|
}
|
||||||
|
|
||||||
private IndexTaskSegmentAllocator createSegmentAllocator(
|
|
||||||
TaskToolbox toolbox,
|
|
||||||
DataSchema dataSchema,
|
|
||||||
Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
if (ingestionSchema.ioConfig.isAppendToExisting() || isUseSegmentLock()) {
|
|
||||||
return new RemoteSegmentAllocator(
|
|
||||||
toolbox,
|
|
||||||
getId(),
|
|
||||||
dataSchema,
|
|
||||||
getSegmentLockHelper(),
|
|
||||||
isUseSegmentLock() ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK,
|
|
||||||
ingestionSchema.ioConfig.isAppendToExisting()
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
// 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(), getId(), getDataSource(), allocateSpec);
|
|
||||||
} else {
|
|
||||||
return new LocalSegmentAllocator(toolbox, getId(), getDataSource(), dataSchema.getGranularitySpec());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
|
* This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
|
||||||
* If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs
|
* If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs
|
||||||
|
@ -856,10 +858,9 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
private TaskStatus generateAndPublishSegments(
|
private TaskStatus generateAndPublishSegments(
|
||||||
final TaskToolbox toolbox,
|
final TaskToolbox toolbox,
|
||||||
final DataSchema dataSchema,
|
final DataSchema dataSchema,
|
||||||
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec,
|
|
||||||
final InputSource inputSource,
|
final InputSource inputSource,
|
||||||
final File tmpDir,
|
final File tmpDir,
|
||||||
final PartitionsSpec partitionsSpec
|
final PartitionAnalysis partitionAnalysis
|
||||||
) throws IOException, InterruptedException
|
) throws IOException, InterruptedException
|
||||||
{
|
{
|
||||||
final FireDepartment fireDepartmentForMetrics =
|
final FireDepartment fireDepartmentForMetrics =
|
||||||
|
@ -875,14 +876,42 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
|
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final PartitionsSpec partitionsSpec = partitionAnalysis.getPartitionsSpec();
|
||||||
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
||||||
final long pushTimeout = tuningConfig.getPushTimeout();
|
final long pushTimeout = tuningConfig.getPushTimeout();
|
||||||
|
|
||||||
final IndexTaskSegmentAllocator segmentAllocator = createSegmentAllocator(
|
final SegmentAllocator segmentAllocator;
|
||||||
|
final SequenceNameFunction sequenceNameFunction;
|
||||||
|
switch (partitionsSpec.getType()) {
|
||||||
|
case HASH:
|
||||||
|
case RANGE:
|
||||||
|
final CachingSegmentAllocator localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning(
|
||||||
toolbox,
|
toolbox,
|
||||||
dataSchema,
|
getDataSource(),
|
||||||
allocateSpec
|
getId(),
|
||||||
|
null,
|
||||||
|
(CompletePartitionAnalysis) partitionAnalysis
|
||||||
);
|
);
|
||||||
|
sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(
|
||||||
|
getId(),
|
||||||
|
localSegmentAllocator.getShardSpecs()
|
||||||
|
);
|
||||||
|
segmentAllocator = localSegmentAllocator;
|
||||||
|
break;
|
||||||
|
case LINEAR:
|
||||||
|
segmentAllocator = SegmentAllocators.forLinearPartitioning(
|
||||||
|
toolbox,
|
||||||
|
null,
|
||||||
|
dataSchema,
|
||||||
|
getTaskLockHelper(),
|
||||||
|
ingestionSchema.getIOConfig().isAppendToExisting(),
|
||||||
|
partitionAnalysis.getPartitionsSpec()
|
||||||
|
);
|
||||||
|
sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(getId());
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw new UOE("[%s] secondary partition type is not supported", partitionsSpec.getType());
|
||||||
|
}
|
||||||
|
|
||||||
final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) ->
|
final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) ->
|
||||||
toolbox.getTaskActionClient()
|
toolbox.getTaskActionClient()
|
||||||
|
@ -921,13 +950,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
inputSource,
|
inputSource,
|
||||||
inputSource.needsFormat() ? getInputFormat(ingestionSchema) : null,
|
inputSource.needsFormat() ? getInputFormat(ingestionSchema) : null,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
segmentAllocator
|
sequenceNameFunction
|
||||||
);
|
);
|
||||||
|
|
||||||
// If we use timeChunk lock, then we don't have to specify what segments will be overwritten because
|
// If we use timeChunk lock, then we don't have to specify what segments will be overwritten because
|
||||||
// it will just overwrite all segments overlapped with the new segments.
|
// it will just overwrite all segments overlapped with the new segments.
|
||||||
final Set<DataSegment> inputSegments = isUseSegmentLock()
|
final Set<DataSegment> inputSegments = getTaskLockHelper().isUseSegmentLock()
|
||||||
? getSegmentLockHelper().getLockedExistingSegments()
|
? getTaskLockHelper().getLockedExistingSegments()
|
||||||
: null;
|
: null;
|
||||||
// Probably we can publish atomicUpdateGroup along with segments.
|
// Probably we can publish atomicUpdateGroup along with segments.
|
||||||
final SegmentsAndCommitMetadata published =
|
final SegmentsAndCommitMetadata published =
|
||||||
|
@ -1207,11 +1236,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
||||||
@Nullable
|
@Nullable
|
||||||
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
|
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
|
||||||
|
|
||||||
static IndexTuningConfig createDefault()
|
|
||||||
{
|
|
||||||
return new IndexTuningConfig();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
private static PartitionsSpec getDefaultPartitionsSpec(
|
private static PartitionsSpec getDefaultPartitionsSpec(
|
||||||
boolean forceGuaranteedRollup,
|
boolean forceGuaranteedRollup,
|
||||||
|
|
|
@ -96,7 +96,7 @@ public class InputSourceProcessor
|
||||||
InputSource inputSource,
|
InputSource inputSource,
|
||||||
@Nullable InputFormat inputFormat,
|
@Nullable InputFormat inputFormat,
|
||||||
File tmpDir,
|
File tmpDir,
|
||||||
IndexTaskSegmentAllocator segmentAllocator
|
SequenceNameFunction sequenceNameFunction
|
||||||
) throws IOException, InterruptedException, ExecutionException, TimeoutException
|
) throws IOException, InterruptedException, ExecutionException, TimeoutException
|
||||||
{
|
{
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -140,7 +140,7 @@ public class InputSourceProcessor
|
||||||
@SuppressWarnings("OptionalGetWithoutIsPresent")
|
@SuppressWarnings("OptionalGetWithoutIsPresent")
|
||||||
final Interval interval = optInterval.get();
|
final Interval interval = optInterval.get();
|
||||||
|
|
||||||
final String sequenceName = segmentAllocator.getSequenceName(interval, inputRow);
|
final String sequenceName = sequenceNameFunction.getSequenceName(interval, inputRow);
|
||||||
final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName);
|
final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName);
|
||||||
|
|
||||||
if (addResult.isOk()) {
|
if (addResult.isOk()) {
|
||||||
|
|
|
@ -0,0 +1,45 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.druid.data.input.InputRow;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This sequence name function should be used for the linear partitioning. Since the segments are created as needed,
|
||||||
|
* this function uses a single sequence name.
|
||||||
|
*
|
||||||
|
* @see org.apache.druid.indexer.partitions.SecondaryPartitionType
|
||||||
|
*/
|
||||||
|
public class LinearlyPartitionedSequenceNameFunction implements SequenceNameFunction
|
||||||
|
{
|
||||||
|
private final String taskId;
|
||||||
|
|
||||||
|
LinearlyPartitionedSequenceNameFunction(String taskId)
|
||||||
|
{
|
||||||
|
this.taskId = taskId;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getSequenceName(Interval interval, InputRow inputRow)
|
||||||
|
{
|
||||||
|
return taskId;
|
||||||
|
}
|
||||||
|
}
|
|
@ -20,6 +20,8 @@
|
||||||
package org.apache.druid.indexing.common.task;
|
package org.apache.druid.indexing.common.task;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import org.apache.commons.lang3.mutable.MutableInt;
|
||||||
import org.apache.druid.data.input.InputRow;
|
import org.apache.druid.data.input.InputRow;
|
||||||
import org.apache.druid.indexing.common.TaskLock;
|
import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
@ -32,32 +34,27 @@ import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Segment allocator which allocates new segments locally per request.
|
* Segment allocator which allocates new segments locally per request.
|
||||||
*/
|
*/
|
||||||
class LocalSegmentAllocator implements IndexTaskSegmentAllocator
|
class LocalSegmentAllocator implements SegmentAllocator
|
||||||
{
|
{
|
||||||
private final String taskId;
|
|
||||||
|
|
||||||
private final SegmentAllocator internalAllocator;
|
private final SegmentAllocator internalAllocator;
|
||||||
|
|
||||||
LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec)
|
LocalSegmentAllocator(TaskToolbox toolbox, String dataSource, GranularitySpec granularitySpec) throws IOException
|
||||||
throws IOException
|
|
||||||
{
|
{
|
||||||
this.taskId = taskId;
|
final Map<Interval, String> intervalToVersion = toolbox
|
||||||
final Map<Interval, AtomicInteger> counters = new HashMap<>();
|
.getTaskActionClient()
|
||||||
|
|
||||||
final Map<Interval, String> intervalToVersion = toolbox.getTaskActionClient()
|
|
||||||
.submit(new LockListAction())
|
.submit(new LockListAction())
|
||||||
.stream()
|
.stream()
|
||||||
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
|
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
|
||||||
|
final Map<Interval, MutableInt> counters = Maps.newHashMapWithExpectedSize(intervalToVersion.size());
|
||||||
|
|
||||||
internalAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
|
internalAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
|
||||||
final DateTime timestamp = row.getTimestamp();
|
final DateTime timestamp = row.getTimestamp();
|
||||||
|
@ -67,30 +64,25 @@ class LocalSegmentAllocator implements IndexTaskSegmentAllocator
|
||||||
}
|
}
|
||||||
|
|
||||||
final Interval interval = maybeInterval.get();
|
final Interval interval = maybeInterval.get();
|
||||||
final String version = intervalToVersion.entrySet().stream()
|
final String version = intervalToVersion
|
||||||
|
.entrySet()
|
||||||
|
.stream()
|
||||||
.filter(entry -> entry.getKey().contains(interval))
|
.filter(entry -> entry.getKey().contains(interval))
|
||||||
.map(Entry::getValue)
|
.map(Entry::getValue)
|
||||||
.findFirst()
|
.findFirst()
|
||||||
.orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
|
.orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
|
||||||
|
|
||||||
final int partitionNum = counters.computeIfAbsent(interval, x -> new AtomicInteger()).getAndIncrement();
|
final int partitionId = counters.computeIfAbsent(interval, x -> new MutableInt()).getAndIncrement();
|
||||||
return new SegmentIdWithShardSpec(
|
return new SegmentIdWithShardSpec(
|
||||||
dataSource,
|
dataSource,
|
||||||
interval,
|
interval,
|
||||||
version,
|
version,
|
||||||
new NumberedShardSpec(partitionNum, 0)
|
new NumberedShardSpec(partitionId, 0)
|
||||||
);
|
);
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Nullable
|
||||||
public String getSequenceName(Interval interval, InputRow inputRow)
|
|
||||||
{
|
|
||||||
// Segments are created as needed, using a single sequence name. They may be allocated from the overlord
|
|
||||||
// (in append mode) or may be created on our own authority (in overwrite mode).
|
|
||||||
return taskId;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SegmentIdWithShardSpec allocate(
|
public SegmentIdWithShardSpec allocate(
|
||||||
InputRow row,
|
InputRow row,
|
||||||
|
|
|
@ -0,0 +1,65 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.druid.data.input.InputRow;
|
||||||
|
import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs;
|
||||||
|
import org.apache.druid.timeline.partition.ShardSpec;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This sequence name function should be used for the hash or range partitioning. This function creates a proper
|
||||||
|
* sequence name based on the partition information (time chunk + partition ID).
|
||||||
|
*
|
||||||
|
* Note that all segment IDs should be allocated upfront to use this function.
|
||||||
|
*
|
||||||
|
* @see org.apache.druid.indexer.partitions.SecondaryPartitionType
|
||||||
|
* @see CachingSegmentAllocator
|
||||||
|
*/
|
||||||
|
public class NonLinearlyPartitionedSequenceNameFunction implements SequenceNameFunction
|
||||||
|
{
|
||||||
|
private final String taskId;
|
||||||
|
private final ShardSpecs shardSpecs;
|
||||||
|
|
||||||
|
public NonLinearlyPartitionedSequenceNameFunction(String taskId, ShardSpecs shardSpecs)
|
||||||
|
{
|
||||||
|
this.taskId = taskId;
|
||||||
|
this.shardSpecs = shardSpecs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getSequenceName(Interval interval, InputRow inputRow)
|
||||||
|
{
|
||||||
|
// Sequence name is based solely on the shardSpec, and there will only be one segment per sequence.
|
||||||
|
return getSequenceName(interval, shardSpecs.getShardSpec(interval, inputRow));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a sequence name from the given shardSpec and interval.
|
||||||
|
*
|
||||||
|
* See {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec} as an example of partitioning.
|
||||||
|
*/
|
||||||
|
private String getSequenceName(Interval interval, ShardSpec shardSpec)
|
||||||
|
{
|
||||||
|
// Note: We do not use String format here since this can be called in a tight loop
|
||||||
|
// and it's faster to add strings together than it is to use String#format
|
||||||
|
return taskId + "_" + interval + "_" + shardSpec.getPartitionNum();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,149 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.druid.indexing.common.task;
|
||||||
|
|
||||||
|
import org.apache.druid.data.input.InputRow;
|
||||||
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import org.apache.druid.indexer.partitions.SecondaryPartitionType;
|
||||||
|
import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
|
||||||
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
|
||||||
|
import org.apache.druid.indexing.common.actions.SurrogateAction;
|
||||||
|
import org.apache.druid.indexing.common.task.TaskLockHelper.OverwritingRootGenerationPartitions;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess;
|
||||||
|
import org.apache.druid.java.util.common.ISE;
|
||||||
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Segment allocator which allocates new segments using the overlord per request.
|
||||||
|
*/
|
||||||
|
public class OverlordCoordinatingSegmentAllocator implements SegmentAllocator
|
||||||
|
{
|
||||||
|
private final ActionBasedSegmentAllocator internalAllocator;
|
||||||
|
|
||||||
|
OverlordCoordinatingSegmentAllocator(
|
||||||
|
final TaskToolbox toolbox,
|
||||||
|
final @Nullable SupervisorTaskAccess supervisorTaskAccess,
|
||||||
|
final DataSchema dataSchema,
|
||||||
|
final TaskLockHelper taskLockHelper,
|
||||||
|
final boolean appendToExisting,
|
||||||
|
final PartitionsSpec partitionsSpec
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.internalAllocator = new ActionBasedSegmentAllocator(
|
||||||
|
toolbox.getTaskActionClient(),
|
||||||
|
dataSchema,
|
||||||
|
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
|
||||||
|
final GranularitySpec granularitySpec = schema.getGranularitySpec();
|
||||||
|
final Interval interval = granularitySpec
|
||||||
|
.bucketInterval(row.getTimestamp())
|
||||||
|
.or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp()));
|
||||||
|
final PartialShardSpec partialShardSpec = createPartialShardSpec(
|
||||||
|
appendToExisting,
|
||||||
|
partitionsSpec,
|
||||||
|
taskLockHelper,
|
||||||
|
interval
|
||||||
|
);
|
||||||
|
if (supervisorTaskAccess != null) {
|
||||||
|
return new SurrogateAction<>(
|
||||||
|
supervisorTaskAccess.getSupervisorTaskId(),
|
||||||
|
new SegmentAllocateAction(
|
||||||
|
schema.getDataSource(),
|
||||||
|
row.getTimestamp(),
|
||||||
|
schema.getGranularitySpec().getQueryGranularity(),
|
||||||
|
schema.getGranularitySpec().getSegmentGranularity(),
|
||||||
|
sequenceName,
|
||||||
|
previousSegmentId,
|
||||||
|
skipSegmentLineageCheck,
|
||||||
|
partialShardSpec,
|
||||||
|
taskLockHelper.getLockGranularityToUse()
|
||||||
|
)
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
return new SegmentAllocateAction(
|
||||||
|
schema.getDataSource(),
|
||||||
|
row.getTimestamp(),
|
||||||
|
schema.getGranularitySpec().getQueryGranularity(),
|
||||||
|
schema.getGranularitySpec().getSegmentGranularity(),
|
||||||
|
sequenceName,
|
||||||
|
previousSegmentId,
|
||||||
|
skipSegmentLineageCheck,
|
||||||
|
partialShardSpec,
|
||||||
|
taskLockHelper.getLockGranularityToUse()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public SegmentIdWithShardSpec allocate(
|
||||||
|
InputRow row,
|
||||||
|
String sequenceName,
|
||||||
|
String previousSegmentId,
|
||||||
|
boolean skipSegmentLineageCheck
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static PartialShardSpec createPartialShardSpec(
|
||||||
|
boolean appendToExisting,
|
||||||
|
PartitionsSpec partitionsSpec,
|
||||||
|
TaskLockHelper taskLockHelper,
|
||||||
|
Interval interval
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (partitionsSpec.getType() == SecondaryPartitionType.LINEAR) {
|
||||||
|
if (taskLockHelper.isUseSegmentLock()) {
|
||||||
|
if (taskLockHelper.hasOverwritingRootGenerationPartition(interval) && !appendToExisting) {
|
||||||
|
final OverwritingRootGenerationPartitions overwritingRootGenerationPartitions = taskLockHelper
|
||||||
|
.getOverwritingRootGenerationPartition(interval);
|
||||||
|
if (overwritingRootGenerationPartitions == null) {
|
||||||
|
throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval);
|
||||||
|
}
|
||||||
|
return new NumberedOverwritePartialShardSpec(
|
||||||
|
overwritingRootGenerationPartitions.getStartRootPartitionId(),
|
||||||
|
overwritingRootGenerationPartitions.getEndRootPartitionId(),
|
||||||
|
overwritingRootGenerationPartitions.getMinorVersionForNewSegments()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return NumberedPartialShardSpec.instance();
|
||||||
|
} else {
|
||||||
|
throw new ISE(
|
||||||
|
"%s is not supported for partitionsSpec[%s]",
|
||||||
|
OverlordCoordinatingSegmentAllocator.class.getName(),
|
||||||
|
partitionsSpec.getClass().getName()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,152 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing,
|
|
||||||
* software distributed under the License is distributed on an
|
|
||||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
||||||
* KIND, either express or implied. See the License for the
|
|
||||||
* specific language governing permissions and limitations
|
|
||||||
* under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.task;
|
|
||||||
|
|
||||||
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);
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,127 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing,
|
|
||||||
* software distributed under the License is distributed on an
|
|
||||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
||||||
* KIND, either express or implied. See the License for the
|
|
||||||
* specific language governing permissions and limitations
|
|
||||||
* under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.task;
|
|
||||||
|
|
||||||
import org.apache.druid.data.input.InputRow;
|
|
||||||
import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
|
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
|
||||||
import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
|
|
||||||
import org.apache.druid.indexing.common.task.SegmentLockHelper.OverwritingRootGenerationPartitions;
|
|
||||||
import org.apache.druid.java.util.common.ISE;
|
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
|
||||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
|
||||||
import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory;
|
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.joda.time.Interval;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Segment allocator which allocates new segments using the overlord per request.
|
|
||||||
*/
|
|
||||||
public class RemoteSegmentAllocator implements IndexTaskSegmentAllocator
|
|
||||||
{
|
|
||||||
private final String taskId;
|
|
||||||
private final ActionBasedSegmentAllocator internalAllocator;
|
|
||||||
|
|
||||||
RemoteSegmentAllocator(
|
|
||||||
final TaskToolbox toolbox,
|
|
||||||
final String taskId,
|
|
||||||
final DataSchema dataSchema,
|
|
||||||
final SegmentLockHelper segmentLockHelper,
|
|
||||||
final LockGranularity lockGranularity,
|
|
||||||
final boolean appendToExisting
|
|
||||||
)
|
|
||||||
{
|
|
||||||
this.taskId = taskId;
|
|
||||||
this.internalAllocator = new ActionBasedSegmentAllocator(
|
|
||||||
toolbox.getTaskActionClient(),
|
|
||||||
dataSchema,
|
|
||||||
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
|
|
||||||
final GranularitySpec granularitySpec = schema.getGranularitySpec();
|
|
||||||
final Interval interval = granularitySpec
|
|
||||||
.bucketInterval(row.getTimestamp())
|
|
||||||
.or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp()));
|
|
||||||
if (lockGranularity == LockGranularity.TIME_CHUNK) {
|
|
||||||
return new SegmentAllocateAction(
|
|
||||||
schema.getDataSource(),
|
|
||||||
row.getTimestamp(),
|
|
||||||
schema.getGranularitySpec().getQueryGranularity(),
|
|
||||||
schema.getGranularitySpec().getSegmentGranularity(),
|
|
||||||
sequenceName,
|
|
||||||
previousSegmentId,
|
|
||||||
skipSegmentLineageCheck,
|
|
||||||
NumberedShardSpecFactory.instance(),
|
|
||||||
lockGranularity
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
final ShardSpecFactory shardSpecFactory;
|
|
||||||
if (segmentLockHelper.hasLockedExistingSegments() && !appendToExisting) {
|
|
||||||
final OverwritingRootGenerationPartitions overwritingSegmentMeta = segmentLockHelper
|
|
||||||
.getOverwritingRootGenerationPartition(interval);
|
|
||||||
if (overwritingSegmentMeta == null) {
|
|
||||||
throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval);
|
|
||||||
}
|
|
||||||
shardSpecFactory = new NumberedOverwritingShardSpecFactory(
|
|
||||||
overwritingSegmentMeta.getStartRootPartitionId(),
|
|
||||||
overwritingSegmentMeta.getEndRootPartitionId(),
|
|
||||||
overwritingSegmentMeta.getMinorVersionForNewSegments()
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
shardSpecFactory = NumberedShardSpecFactory.instance();
|
|
||||||
}
|
|
||||||
return new SegmentAllocateAction(
|
|
||||||
schema.getDataSource(),
|
|
||||||
row.getTimestamp(),
|
|
||||||
schema.getGranularitySpec().getQueryGranularity(),
|
|
||||||
schema.getGranularitySpec().getSegmentGranularity(),
|
|
||||||
sequenceName,
|
|
||||||
previousSegmentId,
|
|
||||||
skipSegmentLineageCheck,
|
|
||||||
shardSpecFactory,
|
|
||||||
lockGranularity
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public SegmentIdWithShardSpec allocate(
|
|
||||||
InputRow row,
|
|
||||||
String sequenceName,
|
|
||||||
String previousSegmentId,
|
|
||||||
boolean skipSegmentLineageCheck
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getSequenceName(Interval interval, InputRow inputRow)
|
|
||||||
{
|
|
||||||
// Segments are created as needed, using a single sequence name. They may be allocated from the overlord
|
|
||||||
// (in append mode) or may be created on our own authority (in overwrite mode).
|
|
||||||
return taskId;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -0,0 +1,96 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis;
|
||||||
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
public final class SegmentAllocators
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* Creates a new {@link SegmentAllocator} for the linear partitioning.
|
||||||
|
* supervisorTaskAccess can be null if this method is called by the {@link IndexTask}.
|
||||||
|
*/
|
||||||
|
public static SegmentAllocator forLinearPartitioning(
|
||||||
|
final TaskToolbox toolbox,
|
||||||
|
final @Nullable SupervisorTaskAccess supervisorTaskAccess,
|
||||||
|
final DataSchema dataSchema,
|
||||||
|
final TaskLockHelper taskLockHelper,
|
||||||
|
final boolean appendToExisting,
|
||||||
|
final PartitionsSpec partitionsSpec
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
if (appendToExisting || taskLockHelper.isUseSegmentLock()) {
|
||||||
|
return new OverlordCoordinatingSegmentAllocator(
|
||||||
|
toolbox,
|
||||||
|
supervisorTaskAccess,
|
||||||
|
dataSchema,
|
||||||
|
taskLockHelper,
|
||||||
|
appendToExisting,
|
||||||
|
partitionsSpec
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
if (supervisorTaskAccess == null) {
|
||||||
|
return new LocalSegmentAllocator(
|
||||||
|
toolbox,
|
||||||
|
dataSchema.getDataSource(),
|
||||||
|
dataSchema.getGranularitySpec()
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
return new SupervisorTaskCoordinatingSegmentAllocator(
|
||||||
|
supervisorTaskAccess.getSupervisorTaskId(),
|
||||||
|
supervisorTaskAccess.getTaskClient()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new {@link SegmentAllocator} for the hash and range partitioning.
|
||||||
|
* supervisorTaskAccess can be null if this method is called by the {@link IndexTask}.
|
||||||
|
*/
|
||||||
|
public static CachingSegmentAllocator forNonLinearPartitioning(
|
||||||
|
final TaskToolbox toolbox,
|
||||||
|
final String dataSource,
|
||||||
|
final String taskId,
|
||||||
|
final @Nullable SupervisorTaskAccess supervisorTaskAccess,
|
||||||
|
final CompletePartitionAnalysis partitionAnalysis
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
return new CachingLocalSegmentAllocator(
|
||||||
|
toolbox,
|
||||||
|
dataSource,
|
||||||
|
taskId,
|
||||||
|
supervisorTaskAccess,
|
||||||
|
partitionAnalysis::convertToIntervalToSegmentIds
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private SegmentAllocators()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
}
|
|
@ -23,20 +23,13 @@ import org.apache.druid.data.input.InputRow;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
/**
|
public interface SequenceNameFunction
|
||||||
* Segment allocator interface for {@link IndexTask}. It has 3 different modes for allocating segments.
|
|
||||||
*/
|
|
||||||
public interface IndexTaskSegmentAllocator extends SegmentAllocator
|
|
||||||
{
|
{
|
||||||
/**
|
/**
|
||||||
* SequenceName is the key to create the segmentId. If previousSegmentId is given, {@link SegmentAllocator} allocates
|
* The sequence name is to keep track of the row sequences that should be stored in the same segment.
|
||||||
* segmentId depending on sequenceName and previousSegmentId. If it's missing, it allocates segmentId using
|
* In other words, given two rows, they should be stored in the same segment if their sequence name is same.
|
||||||
* sequenceName and interval. For {@link IndexTask}, it always provides the previousSegmentId to
|
|
||||||
* SegmentAllocator.
|
|
||||||
* See {@link org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator#allocatePendingSegment} for details.
|
|
||||||
*
|
*
|
||||||
* Implementations should return the correct sequenceName based on the given interval and inputRow, which is passed
|
* @see SegmentAllocator
|
||||||
* to SegmentAllocator.
|
|
||||||
*/
|
*/
|
||||||
String getSequenceName(Interval interval, InputRow inputRow);
|
String getSequenceName(Interval interval, InputRow inputRow);
|
||||||
}
|
}
|
|
@ -0,0 +1,56 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.druid.data.input.InputRow;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Segment allocator that allocates new segments using the supervisor task per request.
|
||||||
|
*/
|
||||||
|
public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAllocator
|
||||||
|
{
|
||||||
|
private final String supervisorTaskId;
|
||||||
|
private final ParallelIndexSupervisorTaskClient taskClient;
|
||||||
|
|
||||||
|
SupervisorTaskCoordinatingSegmentAllocator(
|
||||||
|
String supervisorTaskId,
|
||||||
|
ParallelIndexSupervisorTaskClient taskClient
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.supervisorTaskId = supervisorTaskId;
|
||||||
|
this.taskClient = taskClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SegmentIdWithShardSpec allocate(
|
||||||
|
InputRow row,
|
||||||
|
String sequenceName,
|
||||||
|
String previousSegmentId,
|
||||||
|
boolean skipSegmentLineageCheck
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
return taskClient.allocateSegment(supervisorTaskId, row.getTimestamp());
|
||||||
|
}
|
||||||
|
}
|
|
@ -20,6 +20,7 @@
|
||||||
package org.apache.druid.indexing.common.task;
|
package org.apache.druid.indexing.common.task;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.druid.indexing.common.LockGranularity;
|
||||||
import org.apache.druid.indexing.common.SegmentLock;
|
import org.apache.druid.indexing.common.SegmentLock;
|
||||||
import org.apache.druid.indexing.common.TaskLockType;
|
import org.apache.druid.indexing.common.TaskLockType;
|
||||||
import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction;
|
import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction;
|
||||||
|
@ -46,12 +47,20 @@ import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class provides some methods to use the segment lock easier and caches the information of locked segments.
|
* This class provides 3 functionalities.
|
||||||
|
* - {@link #verifyAndLockExistingSegments} is to verify the granularity of existing segments and lock them.
|
||||||
|
* This method must be called before the task starts indexing.
|
||||||
|
* - Tells the task what {@link LockGranularity} it should use. Note that the LockGranularity is determined in
|
||||||
|
* {@link AbstractBatchIndexTask#determineLockGranularityandTryLock}.
|
||||||
|
* - Provides some util methods for {@link LockGranularity#SEGMENT}. Also caches the information of locked segments when
|
||||||
|
* - the SEGMENt lock granularity is used.
|
||||||
*/
|
*/
|
||||||
public class SegmentLockHelper
|
public class TaskLockHelper
|
||||||
{
|
{
|
||||||
private final Map<Interval, OverwritingRootGenerationPartitions> overwritingRootGenPartitions = new HashMap<>();
|
private final Map<Interval, OverwritingRootGenerationPartitions> overwritingRootGenPartitions = new HashMap<>();
|
||||||
private final Set<DataSegment> lockedExistingSegments = new HashSet<>();
|
private final Set<DataSegment> lockedExistingSegments = new HashSet<>();
|
||||||
|
private final boolean useSegmentLock;
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
private Granularity knownSegmentGranularity;
|
private Granularity knownSegmentGranularity;
|
||||||
|
|
||||||
|
@ -84,6 +93,21 @@ public class SegmentLockHelper
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public TaskLockHelper(boolean useSegmentLock)
|
||||||
|
{
|
||||||
|
this.useSegmentLock = useSegmentLock;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isUseSegmentLock()
|
||||||
|
{
|
||||||
|
return useSegmentLock;
|
||||||
|
}
|
||||||
|
|
||||||
|
public LockGranularity getLockGranularityToUse()
|
||||||
|
{
|
||||||
|
return useSegmentLock ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK;
|
||||||
|
}
|
||||||
|
|
||||||
public boolean hasLockedExistingSegments()
|
public boolean hasLockedExistingSegments()
|
||||||
{
|
{
|
||||||
return !lockedExistingSegments.isEmpty();
|
return !lockedExistingSegments.isEmpty();
|
|
@ -58,7 +58,6 @@ import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.Tasks;
|
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.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.StringDistribution;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistributionMerger;
|
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistributionMerger;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketchMerger;
|
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketchMerger;
|
||||||
|
@ -81,6 +80,7 @@ import org.apache.druid.server.security.Action;
|
||||||
import org.apache.druid.server.security.AuthorizerMapper;
|
import org.apache.druid.server.security.AuthorizerMapper;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.apache.druid.utils.CollectionUtils;
|
import org.apache.druid.utils.CollectionUtils;
|
||||||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.druid.data.input.SplitHintSpec;
|
import org.apache.druid.data.input.SplitHintSpec;
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
|
import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
|
@ -185,6 +186,11 @@ public class ParallelIndexTuningConfig extends IndexTuningConfig
|
||||||
Preconditions.checkArgument(this.maxNumConcurrentSubTasks > 0, "maxNumConcurrentSubTasks must be positive");
|
Preconditions.checkArgument(this.maxNumConcurrentSubTasks > 0, "maxNumConcurrentSubTasks must be positive");
|
||||||
Preconditions.checkArgument(this.maxNumSegmentsToMerge > 0, "maxNumSegmentsToMerge must be positive");
|
Preconditions.checkArgument(this.maxNumSegmentsToMerge > 0, "maxNumSegmentsToMerge must be positive");
|
||||||
Preconditions.checkArgument(this.totalNumMergeTasks > 0, "totalNumMergeTasks must be positive");
|
Preconditions.checkArgument(this.totalNumMergeTasks > 0, "totalNumMergeTasks must be positive");
|
||||||
|
if (getPartitionsSpec() != null && getPartitionsSpec() instanceof SingleDimensionPartitionsSpec) {
|
||||||
|
if (((SingleDimensionPartitionsSpec) getPartitionsSpec()).getPartitionDimension() == null) {
|
||||||
|
throw new IAE("partitionDimension must be specified");
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
|
|
@ -26,22 +26,24 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
|
||||||
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
import org.apache.druid.indexing.common.task.HashPartitionCachingLocalSegmentAllocator;
|
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator;
|
import org.apache.druid.indexing.common.task.SegmentAllocators;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder;
|
import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
|
||||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nonnull;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.SortedSet;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -125,14 +127,18 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException
|
CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
|
||||||
|
throws IOException
|
||||||
{
|
{
|
||||||
return new HashPartitionCachingLocalSegmentAllocator(
|
final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
|
||||||
|
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
||||||
|
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
|
||||||
|
return SegmentAllocators.forNonLinearPartitioning(
|
||||||
toolbox,
|
toolbox,
|
||||||
getId(),
|
|
||||||
supervisorTaskId,
|
|
||||||
getDataSource(),
|
getDataSource(),
|
||||||
createShardSpecs()
|
getId(),
|
||||||
|
new SupervisorTaskAccess(supervisorTaskId, taskClient),
|
||||||
|
createHashPartitionAnalysisFromPartitionsSpec(granularitySpec, partitionsSpec)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -158,17 +164,24 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<Interval, Pair<ShardSpecFactory, Integer>> createShardSpecs()
|
/**
|
||||||
|
* Creates shard specs based on the given configurations. The return value is a map between intervals created
|
||||||
|
* based on the segment granularity and the shard specs to be created.
|
||||||
|
* Note that the shard specs to be created is a pair of {@link PartialShardSpec} and number of segments per interval
|
||||||
|
* and filled only when {@link #isGuaranteedRollup} = true. Otherwise, the return value contains only the set of
|
||||||
|
* intervals generated based on the segment granularity.
|
||||||
|
*/
|
||||||
|
public static HashPartitionAnalysis createHashPartitionAnalysisFromPartitionsSpec(
|
||||||
|
GranularitySpec granularitySpec,
|
||||||
|
@Nonnull HashedPartitionsSpec partitionsSpec
|
||||||
|
)
|
||||||
{
|
{
|
||||||
GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
|
final SortedSet<Interval> intervals = granularitySpec.bucketIntervals().get();
|
||||||
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
|
final int numBucketsPerInterval = partitionsSpec.getNumShards() == null
|
||||||
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
|
? 1
|
||||||
|
: partitionsSpec.getNumShards();
|
||||||
return createShardSpecWithoutInputScan(
|
final HashPartitionAnalysis partitionAnalysis = new HashPartitionAnalysis(partitionsSpec);
|
||||||
granularitySpec,
|
intervals.forEach(interval -> partitionAnalysis.updateBucket(interval, numBucketsPerInterval));
|
||||||
ingestionSchema.getIOConfig(),
|
return partitionAnalysis;
|
||||||
tuningConfig,
|
|
||||||
partitionsSpec
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,8 +24,8 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
|
||||||
import org.apache.druid.data.input.InputSplit;
|
import org.apache.druid.data.input.InputSplit;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
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.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
|
@ -28,15 +28,16 @@ import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
|
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator;
|
import org.apache.druid.indexing.common.task.SegmentAllocators;
|
||||||
import org.apache.druid.indexing.common.task.RangePartitionCachingLocalSegmentAllocator;
|
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
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.common.task.batch.parallel.iterator.RangePartitionIndexTaskInputRowIteratorBuilder;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.RangePartitionAnalysis;
|
||||||
import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher;
|
import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher;
|
||||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -149,15 +150,19 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException
|
CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
|
||||||
|
throws IOException
|
||||||
{
|
{
|
||||||
return new RangePartitionCachingLocalSegmentAllocator(
|
final RangePartitionAnalysis partitionAnalysis = new RangePartitionAnalysis(
|
||||||
|
(SingleDimensionPartitionsSpec) ingestionSchema.getTuningConfig().getPartitionsSpec()
|
||||||
|
);
|
||||||
|
intervalToPartitions.forEach(partitionAnalysis::updateBucket);
|
||||||
|
return SegmentAllocators.forNonLinearPartitioning(
|
||||||
toolbox,
|
toolbox,
|
||||||
getId(),
|
|
||||||
supervisorTaskId,
|
|
||||||
getDataSource(),
|
getDataSource(),
|
||||||
getPartitionDimension(ingestionSchema),
|
getId(),
|
||||||
intervalToPartitions
|
new SupervisorTaskAccess(supervisorTaskId, taskClient),
|
||||||
|
partitionAnalysis
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,10 +28,12 @@ import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters;
|
import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters;
|
||||||
import org.apache.druid.indexing.common.stats.RowIngestionMeters;
|
import org.apache.druid.indexing.common.stats.RowIngestionMeters;
|
||||||
import org.apache.druid.indexing.common.task.BatchAppenderators;
|
import org.apache.druid.indexing.common.task.BatchAppenderators;
|
||||||
|
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
|
||||||
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
|
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator;
|
|
||||||
import org.apache.druid.indexing.common.task.InputSourceProcessor;
|
import org.apache.druid.indexing.common.task.InputSourceProcessor;
|
||||||
|
import org.apache.druid.indexing.common.task.NonLinearlyPartitionedSequenceNameFunction;
|
||||||
|
import org.apache.druid.indexing.common.task.SequenceNameFunction;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.Tasks;
|
import org.apache.druid.indexing.common.task.Tasks;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder;
|
import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder;
|
||||||
|
@ -45,6 +47,7 @@ import org.apache.druid.segment.realtime.RealtimeMetricsMonitor;
|
||||||
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
||||||
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||||
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
@ -117,16 +120,19 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
||||||
ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
|
ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
|
||||||
);
|
);
|
||||||
|
|
||||||
final List<DataSegment> segments = generateSegments(toolbox, inputSource, tmpDir);
|
final List<DataSegment> segments = generateSegments(toolbox, taskClient, inputSource, tmpDir);
|
||||||
taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments));
|
taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments));
|
||||||
|
|
||||||
return TaskStatus.success(getId());
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return {@link IndexTaskSegmentAllocator} suitable for the desired segment partitioning strategy.
|
* @return {@link SegmentAllocator} suitable for the desired segment partitioning strategy.
|
||||||
*/
|
*/
|
||||||
abstract IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException;
|
abstract CachingSegmentAllocator createSegmentAllocator(
|
||||||
|
TaskToolbox toolbox,
|
||||||
|
ParallelIndexSupervisorTaskClient taskClient
|
||||||
|
) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return {@link GeneratedPartitionsReport} suitable for the desired segment partitioning strategy.
|
* @return {@link GeneratedPartitionsReport} suitable for the desired segment partitioning strategy.
|
||||||
|
@ -138,6 +144,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
||||||
|
|
||||||
private List<DataSegment> generateSegments(
|
private List<DataSegment> generateSegments(
|
||||||
final TaskToolbox toolbox,
|
final TaskToolbox toolbox,
|
||||||
|
final ParallelIndexSupervisorTaskClient taskClient,
|
||||||
final InputSource inputSource,
|
final InputSource inputSource,
|
||||||
final File tmpDir
|
final File tmpDir
|
||||||
) throws IOException, InterruptedException, ExecutionException, TimeoutException
|
) throws IOException, InterruptedException, ExecutionException, TimeoutException
|
||||||
|
@ -164,7 +171,11 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
||||||
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
|
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
|
||||||
final long pushTimeout = tuningConfig.getPushTimeout();
|
final long pushTimeout = tuningConfig.getPushTimeout();
|
||||||
|
|
||||||
final IndexTaskSegmentAllocator segmentAllocator = createSegmentAllocator(toolbox);
|
final CachingSegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient);
|
||||||
|
final SequenceNameFunction sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(
|
||||||
|
getId(),
|
||||||
|
segmentAllocator.getShardSpecs()
|
||||||
|
);
|
||||||
|
|
||||||
final Appenderator appenderator = BatchAppenderators.newAppenderator(
|
final Appenderator appenderator = BatchAppenderators.newAppenderator(
|
||||||
getId(),
|
getId(),
|
||||||
|
@ -195,7 +206,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
|
||||||
inputSource,
|
inputSource,
|
||||||
inputSource.needsFormat() ? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema) : null,
|
inputSource.needsFormat() ? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema) : null,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
segmentAllocator
|
sequenceNameFunction
|
||||||
);
|
);
|
||||||
|
|
||||||
return pushed.getSegments();
|
return pushed.getSegments();
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.task.batch.parallel;
|
||||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
import org.apache.druid.client.indexing.IndexingServiceClient;
|
import org.apache.druid.client.indexing.IndexingServiceClient;
|
||||||
|
@ -32,19 +31,14 @@ import org.apache.druid.data.input.InputSource;
|
||||||
import org.apache.druid.data.input.InputSourceReader;
|
import org.apache.druid.data.input.InputSourceReader;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||||
import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
|
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
|
|
||||||
import org.apache.druid.indexing.common.actions.SurrogateAction;
|
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
import org.apache.druid.indexing.common.task.AbstractBatchIndexTask;
|
import org.apache.druid.indexing.common.task.AbstractBatchIndexTask;
|
||||||
import org.apache.druid.indexing.common.task.BatchAppenderators;
|
import org.apache.druid.indexing.common.task.BatchAppenderators;
|
||||||
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
|
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
|
||||||
import org.apache.druid.indexing.common.task.IndexTask;
|
import org.apache.druid.indexing.common.task.IndexTask;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
||||||
import org.apache.druid.indexing.common.task.SegmentLockHelper;
|
import org.apache.druid.indexing.common.task.SegmentAllocators;
|
||||||
import org.apache.druid.indexing.common.task.SegmentLockHelper.OverwritingRootGenerationPartitions;
|
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.Tasks;
|
import org.apache.druid.indexing.common.task.Tasks;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
|
@ -69,14 +63,10 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
|
||||||
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
|
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
|
||||||
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
||||||
import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory;
|
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
|
||||||
import org.apache.druid.timeline.partition.PartitionChunk;
|
import org.apache.druid.timeline.partition.PartitionChunk;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -230,7 +220,7 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
|
||||||
);
|
);
|
||||||
|
|
||||||
// Find inputSegments overshadowed by pushedSegments
|
// Find inputSegments overshadowed by pushedSegments
|
||||||
final Set<DataSegment> allSegments = new HashSet<>(getSegmentLockHelper().getLockedExistingSegments());
|
final Set<DataSegment> allSegments = new HashSet<>(getTaskLockHelper().getLockedExistingSegments());
|
||||||
allSegments.addAll(pushedSegments);
|
allSegments.addAll(pushedSegments);
|
||||||
final VersionedIntervalTimeline<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(allSegments);
|
final VersionedIntervalTimeline<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(allSegments);
|
||||||
final Set<DataSegment> oldSegments = timeline.findFullyOvershadowed()
|
final Set<DataSegment> oldSegments = timeline.findFullyOvershadowed()
|
||||||
|
@ -279,93 +269,6 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
|
|
||||||
{
|
|
||||||
return new WrappingSegmentAllocator(toolbox, taskClient);
|
|
||||||
}
|
|
||||||
|
|
||||||
private class WrappingSegmentAllocator implements SegmentAllocator
|
|
||||||
{
|
|
||||||
private final TaskToolbox toolbox;
|
|
||||||
private final ParallelIndexSupervisorTaskClient taskClient;
|
|
||||||
|
|
||||||
private SegmentAllocator internalAllocator;
|
|
||||||
|
|
||||||
private WrappingSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
|
|
||||||
{
|
|
||||||
this.toolbox = toolbox;
|
|
||||||
this.taskClient = taskClient;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public SegmentIdWithShardSpec allocate(
|
|
||||||
InputRow row,
|
|
||||||
String sequenceName,
|
|
||||||
String previousSegmentId,
|
|
||||||
boolean skipSegmentLineageCheck
|
|
||||||
) throws IOException
|
|
||||||
{
|
|
||||||
if (internalAllocator == null) {
|
|
||||||
internalAllocator = createSegmentAllocator();
|
|
||||||
}
|
|
||||||
return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck);
|
|
||||||
}
|
|
||||||
|
|
||||||
private SegmentAllocator createSegmentAllocator()
|
|
||||||
{
|
|
||||||
final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
|
|
||||||
final SegmentLockHelper segmentLockHelper = getSegmentLockHelper();
|
|
||||||
if (ingestionSchema.getIOConfig().isAppendToExisting() || isUseSegmentLock()) {
|
|
||||||
return new ActionBasedSegmentAllocator(
|
|
||||||
toolbox.getTaskActionClient(),
|
|
||||||
ingestionSchema.getDataSchema(),
|
|
||||||
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
|
|
||||||
final Interval interval = granularitySpec
|
|
||||||
.bucketInterval(row.getTimestamp())
|
|
||||||
.or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp()));
|
|
||||||
|
|
||||||
final ShardSpecFactory shardSpecFactory;
|
|
||||||
if (segmentLockHelper.hasOverwritingRootGenerationPartition(interval)) {
|
|
||||||
final OverwritingRootGenerationPartitions overwritingSegmentMeta = segmentLockHelper
|
|
||||||
.getOverwritingRootGenerationPartition(interval);
|
|
||||||
if (overwritingSegmentMeta == null) {
|
|
||||||
throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval);
|
|
||||||
}
|
|
||||||
shardSpecFactory = new NumberedOverwritingShardSpecFactory(
|
|
||||||
overwritingSegmentMeta.getStartRootPartitionId(),
|
|
||||||
overwritingSegmentMeta.getEndRootPartitionId(),
|
|
||||||
overwritingSegmentMeta.getMinorVersionForNewSegments()
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
shardSpecFactory = NumberedShardSpecFactory.instance();
|
|
||||||
}
|
|
||||||
|
|
||||||
return new SurrogateAction<>(
|
|
||||||
supervisorTaskId,
|
|
||||||
new SegmentAllocateAction(
|
|
||||||
schema.getDataSource(),
|
|
||||||
row.getTimestamp(),
|
|
||||||
schema.getGranularitySpec().getQueryGranularity(),
|
|
||||||
schema.getGranularitySpec().getSegmentGranularity(),
|
|
||||||
sequenceName,
|
|
||||||
previousSegmentId,
|
|
||||||
skipSegmentLineageCheck,
|
|
||||||
shardSpecFactory,
|
|
||||||
isUseSegmentLock() ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
return (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> taskClient.allocateSegment(
|
|
||||||
supervisorTaskId,
|
|
||||||
row.getTimestamp()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
|
* This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
|
||||||
* If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs
|
* If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs
|
||||||
|
@ -410,7 +313,14 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
|
||||||
final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
|
final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
|
||||||
final long pushTimeout = tuningConfig.getPushTimeout();
|
final long pushTimeout = tuningConfig.getPushTimeout();
|
||||||
final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent();
|
final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent();
|
||||||
final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient);
|
final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning(
|
||||||
|
toolbox,
|
||||||
|
new SupervisorTaskAccess(getSupervisorTaskId(), taskClient),
|
||||||
|
getIngestionSchema().getDataSchema(),
|
||||||
|
getTaskLockHelper(),
|
||||||
|
ingestionSchema.getIOConfig().isAppendToExisting(),
|
||||||
|
partitionsSpec
|
||||||
|
);
|
||||||
|
|
||||||
final Appenderator appenderator = BatchAppenderators.newAppenderator(
|
final Appenderator appenderator = BatchAppenderators.newAppenderator(
|
||||||
getId(),
|
getId(),
|
||||||
|
|
|
@ -0,0 +1,50 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Simple POJO to hold the information of the supervisor task.
|
||||||
|
*
|
||||||
|
* @see ParallelIndexSupervisorTask
|
||||||
|
*/
|
||||||
|
public class SupervisorTaskAccess
|
||||||
|
{
|
||||||
|
private final String supervisorTaskId;
|
||||||
|
private final ParallelIndexSupervisorTaskClient taskClient;
|
||||||
|
|
||||||
|
public SupervisorTaskAccess(
|
||||||
|
String supervisorTaskId,
|
||||||
|
ParallelIndexSupervisorTaskClient taskClient
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.supervisorTaskId = supervisorTaskId;
|
||||||
|
this.taskClient = taskClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getSupervisorTaskId()
|
||||||
|
{
|
||||||
|
return supervisorTaskId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ParallelIndexSupervisorTaskClient getTaskClient()
|
||||||
|
{
|
||||||
|
return taskClient;
|
||||||
|
}
|
||||||
|
}
|
|
@ -21,6 +21,7 @@ package org.apache.druid.indexing.common.task.batch.parallel.distribution;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Counts frequencies of {@link String}s.
|
* Counts frequencies of {@link String}s.
|
||||||
|
|
|
@ -34,6 +34,7 @@ import com.google.common.base.Preconditions;
|
||||||
import org.apache.datasketches.ArrayOfStringsSerDe;
|
import org.apache.datasketches.ArrayOfStringsSerDe;
|
||||||
import org.apache.datasketches.memory.Memory;
|
import org.apache.datasketches.memory.Memory;
|
||||||
import org.apache.datasketches.quantiles.ItemsSketch;
|
import org.apache.datasketches.quantiles.ItemsSketch;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
|
@ -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.partition;
|
||||||
|
|
||||||
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This interface represents the PartitionAnalysis that has the complete picture of secondary partitions to create.
|
||||||
|
* This type of PartitionAnalysis can be used for the hash or range partitioning in which all secondary partitions
|
||||||
|
* should be determined when the analysis is done.
|
||||||
|
*/
|
||||||
|
public interface CompletePartitionAnalysis<T, P extends PartitionsSpec> extends PartitionAnalysis<T, P>
|
||||||
|
{
|
||||||
|
Map<Interval, List<SegmentIdWithShardSpec>> convertToIntervalToSegmentIds(
|
||||||
|
TaskToolbox toolbox,
|
||||||
|
String dataSource,
|
||||||
|
Function<Interval, String> versionFinder
|
||||||
|
);
|
||||||
|
}
|
|
@ -0,0 +1,128 @@
|
||||||
|
/*
|
||||||
|
* 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.partition;
|
||||||
|
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
import org.apache.druid.java.util.common.IAE;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.function.BiConsumer;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
|
public class HashPartitionAnalysis implements CompletePartitionAnalysis<Integer, HashedPartitionsSpec>
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* Key is the time ranges for the primary partitioning.
|
||||||
|
* Value is the number of partitions per time range for the secondary partitioning
|
||||||
|
*/
|
||||||
|
private final Map<Interval, Integer> intervalToNumBuckets = new HashMap<>();
|
||||||
|
private final HashedPartitionsSpec partitionsSpec;
|
||||||
|
|
||||||
|
public HashPartitionAnalysis(HashedPartitionsSpec partitionsSpec)
|
||||||
|
{
|
||||||
|
this.partitionsSpec = partitionsSpec;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public HashedPartitionsSpec getPartitionsSpec()
|
||||||
|
{
|
||||||
|
return partitionsSpec;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateBucket(Interval interval, Integer bucketAnalysis)
|
||||||
|
{
|
||||||
|
intervalToNumBuckets.put(interval, bucketAnalysis);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Integer getBucketAnalysis(Interval interval)
|
||||||
|
{
|
||||||
|
final Integer bucketAnalysis = intervalToNumBuckets.get(interval);
|
||||||
|
if (bucketAnalysis == null) {
|
||||||
|
throw new IAE("Missing bucket analysis for interval[%s]", interval);
|
||||||
|
} else {
|
||||||
|
return bucketAnalysis;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<Interval> getAllIntervalsToIndex()
|
||||||
|
{
|
||||||
|
return Collections.unmodifiableSet(intervalToNumBuckets.keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getNumTimePartitions()
|
||||||
|
{
|
||||||
|
return intervalToNumBuckets.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void forEach(BiConsumer<Interval, Integer> consumer)
|
||||||
|
{
|
||||||
|
intervalToNumBuckets.forEach(consumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Interval, List<SegmentIdWithShardSpec>> convertToIntervalToSegmentIds(
|
||||||
|
TaskToolbox toolbox,
|
||||||
|
String dataSource,
|
||||||
|
Function<Interval, String> versionFinder
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToSegmentIds =
|
||||||
|
Maps.newHashMapWithExpectedSize(getNumTimePartitions());
|
||||||
|
|
||||||
|
forEach((interval, numBuckets) -> {
|
||||||
|
intervalToSegmentIds.put(
|
||||||
|
interval,
|
||||||
|
IntStream.range(0, numBuckets)
|
||||||
|
.mapToObj(i -> {
|
||||||
|
final HashBasedNumberedShardSpec shardSpec = new HashBasedNumberedShardSpec(
|
||||||
|
i,
|
||||||
|
numBuckets,
|
||||||
|
partitionsSpec.getPartitionDimensions(),
|
||||||
|
toolbox.getJsonMapper()
|
||||||
|
);
|
||||||
|
return new SegmentIdWithShardSpec(
|
||||||
|
dataSource,
|
||||||
|
interval,
|
||||||
|
versionFinder.apply(interval),
|
||||||
|
shardSpec
|
||||||
|
);
|
||||||
|
})
|
||||||
|
.collect(Collectors.toList())
|
||||||
|
);
|
||||||
|
});
|
||||||
|
|
||||||
|
return intervalToSegmentIds;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,78 @@
|
||||||
|
/*
|
||||||
|
* 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.partition;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||||
|
import org.apache.druid.java.util.common.IAE;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Partition analysis for the linear partitioning. This analysis is not complete because, in the linear partitioning,
|
||||||
|
* segments are partitioned by their size which means they will be allocated dynamically during the indexing.
|
||||||
|
*/
|
||||||
|
public class LinearPartitionAnalysis implements PartitionAnalysis<Integer, DynamicPartitionsSpec>
|
||||||
|
{
|
||||||
|
private final Set<Interval> intervals = new HashSet<>();
|
||||||
|
private final DynamicPartitionsSpec partitionsSpec;
|
||||||
|
|
||||||
|
public LinearPartitionAnalysis(DynamicPartitionsSpec partitionsSpec)
|
||||||
|
{
|
||||||
|
this.partitionsSpec = partitionsSpec;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DynamicPartitionsSpec getPartitionsSpec()
|
||||||
|
{
|
||||||
|
return partitionsSpec;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateBucket(Interval interval, Integer bucketAnalysis)
|
||||||
|
{
|
||||||
|
Preconditions.checkArgument(bucketAnalysis == 1, "There should be only one bucket with linear partitioining");
|
||||||
|
intervals.add(interval);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Integer getBucketAnalysis(Interval interval)
|
||||||
|
{
|
||||||
|
if (intervals.contains(interval)) {
|
||||||
|
return 1;
|
||||||
|
} else {
|
||||||
|
throw new IAE("Missing bucket analysis for interval[%s]", interval);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<Interval> getAllIntervalsToIndex()
|
||||||
|
{
|
||||||
|
return intervals;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getNumTimePartitions()
|
||||||
|
{
|
||||||
|
return intervals.size();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,50 @@
|
||||||
|
/*
|
||||||
|
* 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.partition;
|
||||||
|
|
||||||
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Analysis of the partitions to create. The implementation is mutable and updated by the indexing
|
||||||
|
* {@link org.apache.druid.indexing.common.task.Task}.
|
||||||
|
*
|
||||||
|
* This interface provides all time chunks for the primary partitioning and the bucket information per time chunk
|
||||||
|
* for the secondary partitioning.
|
||||||
|
*/
|
||||||
|
public interface PartitionAnalysis<T, P extends PartitionsSpec>
|
||||||
|
{
|
||||||
|
P getPartitionsSpec();
|
||||||
|
|
||||||
|
void updateBucket(Interval interval, T bucketAnalysis);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the analysis of the secondary bucket for the given time chunk.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the bucket analysis is missing for the given interval
|
||||||
|
*/
|
||||||
|
T getBucketAnalysis(Interval interval);
|
||||||
|
|
||||||
|
Set<Interval> getAllIntervalsToIndex();
|
||||||
|
|
||||||
|
int getNumTimePartitions();
|
||||||
|
}
|
|
@ -0,0 +1,174 @@
|
||||||
|
/*
|
||||||
|
* 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.partition;
|
||||||
|
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
import org.apache.druid.java.util.common.IAE;
|
||||||
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
|
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.function.BiConsumer;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
|
public class RangePartitionAnalysis
|
||||||
|
implements CompletePartitionAnalysis<PartitionBoundaries, SingleDimensionPartitionsSpec>
|
||||||
|
{
|
||||||
|
private final Map<Interval, PartitionBoundaries> intervalToPartitionBoundaries = new HashMap<>();
|
||||||
|
private final SingleDimensionPartitionsSpec partitionsSpec;
|
||||||
|
|
||||||
|
public RangePartitionAnalysis(SingleDimensionPartitionsSpec partitionsSpec)
|
||||||
|
{
|
||||||
|
this.partitionsSpec = partitionsSpec;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SingleDimensionPartitionsSpec getPartitionsSpec()
|
||||||
|
{
|
||||||
|
return partitionsSpec;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateBucket(Interval interval, PartitionBoundaries bucketAnalysis)
|
||||||
|
{
|
||||||
|
intervalToPartitionBoundaries.put(interval, bucketAnalysis);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PartitionBoundaries getBucketAnalysis(Interval interval)
|
||||||
|
{
|
||||||
|
final PartitionBoundaries bucketAnalysis = intervalToPartitionBoundaries.get(interval);
|
||||||
|
if (bucketAnalysis == null) {
|
||||||
|
throw new IAE("Missing bucket analysis for interval[%s]", interval);
|
||||||
|
} else {
|
||||||
|
return bucketAnalysis;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<Interval> getAllIntervalsToIndex()
|
||||||
|
{
|
||||||
|
return Collections.unmodifiableSet(intervalToPartitionBoundaries.keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void forEach(BiConsumer<Interval, PartitionBoundaries> consumer)
|
||||||
|
{
|
||||||
|
intervalToPartitionBoundaries.forEach(consumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getNumTimePartitions()
|
||||||
|
{
|
||||||
|
return intervalToPartitionBoundaries.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Interval, List<SegmentIdWithShardSpec>> convertToIntervalToSegmentIds(
|
||||||
|
TaskToolbox toolbox,
|
||||||
|
String dataSource,
|
||||||
|
Function<Interval, String> versionFinder
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final String partitionDimension = partitionsSpec.getPartitionDimension();
|
||||||
|
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToSegmentIds = Maps.newHashMapWithExpectedSize(
|
||||||
|
getNumTimePartitions()
|
||||||
|
);
|
||||||
|
|
||||||
|
forEach((interval, partitionBoundaries) ->
|
||||||
|
intervalToSegmentIds.put(
|
||||||
|
interval,
|
||||||
|
translatePartitionBoundaries(
|
||||||
|
dataSource,
|
||||||
|
interval,
|
||||||
|
partitionDimension,
|
||||||
|
partitionBoundaries,
|
||||||
|
versionFinder
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
return intervalToSegmentIds;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Translate {@link PartitionBoundaries} into the corresponding
|
||||||
|
* {@link SingleDimensionPartitionsSpec} with segment id.
|
||||||
|
*/
|
||||||
|
private static List<SegmentIdWithShardSpec> translatePartitionBoundaries(
|
||||||
|
String dataSource,
|
||||||
|
Interval interval,
|
||||||
|
String partitionDimension,
|
||||||
|
PartitionBoundaries partitionBoundaries,
|
||||||
|
Function<Interval, String> versionFinder
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (partitionBoundaries.isEmpty()) {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
|
||||||
|
return IntStream.range(0, partitionBoundaries.size() - 1)
|
||||||
|
.mapToObj(i -> createSegmentIdWithShardSpec(
|
||||||
|
dataSource,
|
||||||
|
interval,
|
||||||
|
versionFinder.apply(interval),
|
||||||
|
partitionDimension,
|
||||||
|
partitionBoundaries.get(i),
|
||||||
|
partitionBoundaries.get(i + 1),
|
||||||
|
i
|
||||||
|
))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static SegmentIdWithShardSpec createSegmentIdWithShardSpec(
|
||||||
|
String dataSource,
|
||||||
|
Interval interval,
|
||||||
|
String version,
|
||||||
|
String partitionDimension,
|
||||||
|
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
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -25,7 +25,7 @@ import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskLockType;
|
import org.apache.druid.indexing.common.TaskLockType;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -37,7 +37,7 @@ public class LockRequestForNewSegment implements LockRequest
|
||||||
private final String groupId;
|
private final String groupId;
|
||||||
private final String dataSource;
|
private final String dataSource;
|
||||||
private final Interval interval;
|
private final Interval interval;
|
||||||
private final ShardSpecFactory shardSpecFactory;
|
private final PartialShardSpec partialShardSpec;
|
||||||
private final int priority;
|
private final int priority;
|
||||||
private final String sequenceName;
|
private final String sequenceName;
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -52,7 +52,7 @@ public class LockRequestForNewSegment implements LockRequest
|
||||||
String groupId,
|
String groupId,
|
||||||
String dataSource,
|
String dataSource,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
ShardSpecFactory shardSpecFactory,
|
PartialShardSpec partialShardSpec,
|
||||||
int priority,
|
int priority,
|
||||||
String sequenceName,
|
String sequenceName,
|
||||||
@Nullable String previsousSegmentId,
|
@Nullable String previsousSegmentId,
|
||||||
|
@ -64,7 +64,7 @@ public class LockRequestForNewSegment implements LockRequest
|
||||||
this.groupId = groupId;
|
this.groupId = groupId;
|
||||||
this.dataSource = dataSource;
|
this.dataSource = dataSource;
|
||||||
this.interval = interval;
|
this.interval = interval;
|
||||||
this.shardSpecFactory = shardSpecFactory;
|
this.partialShardSpec = partialShardSpec;
|
||||||
this.priority = priority;
|
this.priority = priority;
|
||||||
this.sequenceName = sequenceName;
|
this.sequenceName = sequenceName;
|
||||||
this.previsousSegmentId = previsousSegmentId;
|
this.previsousSegmentId = previsousSegmentId;
|
||||||
|
@ -77,7 +77,7 @@ public class LockRequestForNewSegment implements LockRequest
|
||||||
TaskLockType lockType,
|
TaskLockType lockType,
|
||||||
Task task,
|
Task task,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
ShardSpecFactory shardSpecFactory,
|
PartialShardSpec partialShardSpec,
|
||||||
String sequenceName,
|
String sequenceName,
|
||||||
@Nullable String previsousSegmentId,
|
@Nullable String previsousSegmentId,
|
||||||
boolean skipSegmentLineageCheck
|
boolean skipSegmentLineageCheck
|
||||||
|
@ -89,7 +89,7 @@ public class LockRequestForNewSegment implements LockRequest
|
||||||
task.getGroupId(),
|
task.getGroupId(),
|
||||||
task.getDataSource(),
|
task.getDataSource(),
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
task.getPriority(),
|
task.getPriority(),
|
||||||
sequenceName,
|
sequenceName,
|
||||||
previsousSegmentId,
|
previsousSegmentId,
|
||||||
|
@ -133,9 +133,9 @@ public class LockRequestForNewSegment implements LockRequest
|
||||||
return priority;
|
return priority;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ShardSpecFactory getShardSpecFactory()
|
public PartialShardSpec getPartialShardSpec()
|
||||||
{
|
{
|
||||||
return shardSpecFactory;
|
return partialShardSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -187,7 +187,7 @@ public class LockRequestForNewSegment implements LockRequest
|
||||||
", groupId='" + groupId + '\'' +
|
", groupId='" + groupId + '\'' +
|
||||||
", dataSource='" + dataSource + '\'' +
|
", dataSource='" + dataSource + '\'' +
|
||||||
", interval=" + interval +
|
", interval=" + interval +
|
||||||
", shardSpecFactory=" + shardSpecFactory +
|
", partialShardSpec=" + partialShardSpec +
|
||||||
", priority=" + priority +
|
", priority=" + priority +
|
||||||
", sequenceName='" + sequenceName + '\'' +
|
", sequenceName='" + sequenceName + '\'' +
|
||||||
", previsousSegmentId='" + previsousSegmentId + '\'' +
|
", previsousSegmentId='" + previsousSegmentId + '\'' +
|
||||||
|
|
|
@ -534,7 +534,7 @@ public class TaskLockbox
|
||||||
request.getSequenceName(),
|
request.getSequenceName(),
|
||||||
request.getPrevisousSegmentId(),
|
request.getPrevisousSegmentId(),
|
||||||
request.getInterval(),
|
request.getInterval(),
|
||||||
request.getShardSpecFactory(),
|
request.getPartialShardSpec(),
|
||||||
version,
|
version,
|
||||||
request.isSkipSegmentLineageCheck()
|
request.isSkipSegmentLineageCheck()
|
||||||
);
|
);
|
||||||
|
|
|
@ -54,7 +54,7 @@ import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
|
||||||
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
import org.apache.druid.segment.realtime.firehose.ChatHandler;
|
||||||
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
|
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
|
||||||
import org.apache.druid.server.security.AuthorizerMapper;
|
import org.apache.druid.server.security.AuthorizerMapper;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
import org.apache.druid.utils.CircularBuffer;
|
import org.apache.druid.utils.CircularBuffer;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -232,7 +232,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
|
||||||
sequenceName,
|
sequenceName,
|
||||||
previousSegmentId,
|
previousSegmentId,
|
||||||
skipSegmentLineageCheck,
|
skipSegmentLineageCheck,
|
||||||
NumberedShardSpecFactory.instance(),
|
NumberedPartialShardSpec.instance(),
|
||||||
lockGranularityToUse
|
lockGranularityToUse
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
|
|
|
@ -0,0 +1,102 @@
|
||||||
|
/*
|
||||||
|
* 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.actions;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.apache.druid.indexing.common.LockGranularity;
|
||||||
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
|
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.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class SegmentAllocateActionSerdeTest
|
||||||
|
{
|
||||||
|
private final ObjectMapper objectMapper;
|
||||||
|
private final SegmentAllocateAction target;
|
||||||
|
|
||||||
|
public SegmentAllocateActionSerdeTest()
|
||||||
|
{
|
||||||
|
objectMapper = new DefaultObjectMapper();
|
||||||
|
objectMapper.registerSubtypes(NumberedPartialShardSpec.class);
|
||||||
|
|
||||||
|
target = new SegmentAllocateAction(
|
||||||
|
"datasource",
|
||||||
|
DateTimes.nowUtc(),
|
||||||
|
Granularities.MINUTE,
|
||||||
|
Granularities.HOUR,
|
||||||
|
"s1",
|
||||||
|
"prev",
|
||||||
|
false,
|
||||||
|
NumberedPartialShardSpec.instance(),
|
||||||
|
LockGranularity.SEGMENT
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSerde() throws Exception
|
||||||
|
{
|
||||||
|
final SegmentAllocateAction fromJson = (SegmentAllocateAction) objectMapper.readValue(
|
||||||
|
objectMapper.writeValueAsBytes(target),
|
||||||
|
TaskAction.class
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertEquals(target.getDataSource(), fromJson.getDataSource());
|
||||||
|
Assert.assertEquals(target.getTimestamp(), fromJson.getTimestamp());
|
||||||
|
Assert.assertEquals(target.getQueryGranularity(), fromJson.getQueryGranularity());
|
||||||
|
Assert.assertEquals(target.getPreferredSegmentGranularity(), fromJson.getPreferredSegmentGranularity());
|
||||||
|
Assert.assertEquals(target.getSequenceName(), fromJson.getSequenceName());
|
||||||
|
Assert.assertEquals(target.getPreviousSegmentId(), fromJson.getPreviousSegmentId());
|
||||||
|
Assert.assertEquals(target.isSkipSegmentLineageCheck(), fromJson.isSkipSegmentLineageCheck());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testJsonPropertyNames() throws IOException
|
||||||
|
{
|
||||||
|
final Map<String, Object> fromJson = objectMapper.readValue(
|
||||||
|
objectMapper.writeValueAsBytes(target),
|
||||||
|
Map.class
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertEquals(10, fromJson.size());
|
||||||
|
Assert.assertEquals(SegmentAllocateAction.TYPE, fromJson.get("type"));
|
||||||
|
Assert.assertEquals(target.getDataSource(), fromJson.get("dataSource"));
|
||||||
|
Assert.assertEquals(target.getTimestamp(), DateTimes.of((String) fromJson.get("timestamp")));
|
||||||
|
Assert.assertEquals(
|
||||||
|
target.getQueryGranularity(),
|
||||||
|
Granularity.fromString((String) fromJson.get("queryGranularity"))
|
||||||
|
);
|
||||||
|
Assert.assertEquals(
|
||||||
|
target.getPreferredSegmentGranularity(),
|
||||||
|
Granularity.fromString((String) fromJson.get("preferredSegmentGranularity"))
|
||||||
|
);
|
||||||
|
Assert.assertEquals(target.getSequenceName(), fromJson.get("sequenceName"));
|
||||||
|
Assert.assertEquals(target.getPreviousSegmentId(), fromJson.get("previousSegmentId"));
|
||||||
|
Assert.assertEquals(target.isSkipSegmentLineageCheck(), fromJson.get("skipSegmentLineageCheck"));
|
||||||
|
Assert.assertEquals(ImmutableMap.of("type", "numbered"), fromJson.get("shardSpecFactory"));
|
||||||
|
Assert.assertEquals(target.getLockGranularity(), LockGranularity.valueOf((String) fromJson.get("lockGranularity")));
|
||||||
|
}
|
||||||
|
}
|
|
@ -38,14 +38,14 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.LinearPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||||
import org.apache.druid.timeline.partition.LinearShardSpecFactory;
|
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpec;
|
import org.apache.druid.timeline.partition.ShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
|
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -652,7 +652,7 @@ public class SegmentAllocateActionTest
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
"s1",
|
"s1",
|
||||||
null,
|
null,
|
||||||
LinearShardSpecFactory.instance()
|
LinearPartialShardSpec.instance()
|
||||||
);
|
);
|
||||||
final SegmentIdWithShardSpec id2 = allocate(
|
final SegmentIdWithShardSpec id2 = allocate(
|
||||||
task,
|
task,
|
||||||
|
@ -661,7 +661,7 @@ public class SegmentAllocateActionTest
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
"s1",
|
"s1",
|
||||||
id1.toString(),
|
id1.toString(),
|
||||||
LinearShardSpecFactory.instance()
|
LinearPartialShardSpec.instance()
|
||||||
);
|
);
|
||||||
|
|
||||||
assertSameIdentifier(
|
assertSameIdentifier(
|
||||||
|
@ -899,39 +899,7 @@ public class SegmentAllocateActionTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSerde() throws Exception
|
public void testWithPartialShardSpecAndOvershadowingSegments() throws IOException
|
||||||
{
|
|
||||||
final ObjectMapper objectMapper = new DefaultObjectMapper();
|
|
||||||
objectMapper.registerSubtypes(NumberedShardSpecFactory.class);
|
|
||||||
|
|
||||||
final SegmentAllocateAction action = new SegmentAllocateAction(
|
|
||||||
DATA_SOURCE,
|
|
||||||
PARTY_TIME,
|
|
||||||
Granularities.MINUTE,
|
|
||||||
Granularities.HOUR,
|
|
||||||
"s1",
|
|
||||||
"prev",
|
|
||||||
false,
|
|
||||||
NumberedShardSpecFactory.instance(),
|
|
||||||
lockGranularity
|
|
||||||
);
|
|
||||||
|
|
||||||
final SegmentAllocateAction action2 = (SegmentAllocateAction) objectMapper.readValue(
|
|
||||||
objectMapper.writeValueAsBytes(action),
|
|
||||||
TaskAction.class
|
|
||||||
);
|
|
||||||
|
|
||||||
Assert.assertEquals(action.getDataSource(), action2.getDataSource());
|
|
||||||
Assert.assertEquals(action.getTimestamp(), action2.getTimestamp());
|
|
||||||
Assert.assertEquals(action.getQueryGranularity(), action2.getQueryGranularity());
|
|
||||||
Assert.assertEquals(action.getPreferredSegmentGranularity(), action2.getPreferredSegmentGranularity());
|
|
||||||
Assert.assertEquals(action.getSequenceName(), action2.getSequenceName());
|
|
||||||
Assert.assertEquals(action.getPreviousSegmentId(), action2.getPreviousSegmentId());
|
|
||||||
Assert.assertEquals(action.isSkipSegmentLineageCheck(), action2.isSkipSegmentLineageCheck());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testWithShardSpecFactoryAndOvershadowingSegments() throws IOException
|
|
||||||
{
|
{
|
||||||
final Task task = NoopTask.create();
|
final Task task = NoopTask.create();
|
||||||
taskActionTestKit.getTaskLockbox().add(task);
|
taskActionTestKit.getTaskLockbox().add(task);
|
||||||
|
@ -965,7 +933,7 @@ public class SegmentAllocateActionTest
|
||||||
"seq",
|
"seq",
|
||||||
null,
|
null,
|
||||||
true,
|
true,
|
||||||
new HashBasedNumberedShardSpecFactory(ImmutableList.of("dim1"), 2),
|
new HashBasedNumberedPartialShardSpec(ImmutableList.of("dim1"), 2),
|
||||||
lockGranularity
|
lockGranularity
|
||||||
);
|
);
|
||||||
final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox());
|
final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox());
|
||||||
|
@ -996,7 +964,7 @@ public class SegmentAllocateActionTest
|
||||||
preferredSegmentGranularity,
|
preferredSegmentGranularity,
|
||||||
sequenceName,
|
sequenceName,
|
||||||
sequencePreviousId,
|
sequencePreviousId,
|
||||||
NumberedShardSpecFactory.instance()
|
NumberedPartialShardSpec.instance()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1007,7 +975,7 @@ public class SegmentAllocateActionTest
|
||||||
final Granularity preferredSegmentGranularity,
|
final Granularity preferredSegmentGranularity,
|
||||||
final String sequenceName,
|
final String sequenceName,
|
||||||
final String sequencePreviousId,
|
final String sequencePreviousId,
|
||||||
final ShardSpecFactory shardSpecFactory
|
final PartialShardSpec partialShardSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final SegmentAllocateAction action = new SegmentAllocateAction(
|
final SegmentAllocateAction action = new SegmentAllocateAction(
|
||||||
|
@ -1018,7 +986,7 @@ public class SegmentAllocateActionTest
|
||||||
sequenceName,
|
sequenceName,
|
||||||
sequencePreviousId,
|
sequencePreviousId,
|
||||||
false,
|
false,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
lockGranularity
|
lockGranularity
|
||||||
);
|
);
|
||||||
return action.perform(task, taskActionTestKit.getTaskActionToolbox());
|
return action.perform(task, taskActionTestKit.getTaskActionToolbox());
|
||||||
|
|
|
@ -236,7 +236,7 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true),
|
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -386,7 +386,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true),
|
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -726,7 +726,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
||||||
),
|
),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
IndexTaskTest.createTuningConfig(5000000, null, null, Long.MAX_VALUE, null, null, false, true)
|
IndexTaskTest.createTuningConfig(5000000, null, null, Long.MAX_VALUE, null, false, true)
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
|
@ -796,7 +796,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true),
|
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
|
||||||
appendToExisting
|
appendToExisting
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -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;
|
||||||
|
|
||||||
|
import org.apache.druid.indexing.common.actions.LocalTaskActionClient;
|
||||||
|
import org.apache.druid.indexing.common.actions.SurrogateAction;
|
||||||
|
import org.apache.druid.indexing.common.actions.TaskAction;
|
||||||
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
|
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
|
||||||
|
import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
|
||||||
|
import org.apache.druid.indexing.overlord.TaskStorage;
|
||||||
|
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
|
public class CountingLocalTaskActionClientForTest implements TaskActionClient
|
||||||
|
{
|
||||||
|
private final ConcurrentHashMap<Class<? extends TaskAction>, AtomicInteger> actionCountMap =
|
||||||
|
new ConcurrentHashMap<>();
|
||||||
|
private final LocalTaskActionClient delegate;
|
||||||
|
|
||||||
|
public CountingLocalTaskActionClientForTest(
|
||||||
|
Task task,
|
||||||
|
TaskStorage storage,
|
||||||
|
TaskActionToolbox toolbox
|
||||||
|
)
|
||||||
|
{
|
||||||
|
delegate = new LocalTaskActionClient(task, storage, toolbox, new TaskAuditLogConfig(false));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public <RetType> RetType submit(TaskAction<RetType> taskAction)
|
||||||
|
{
|
||||||
|
final RetType result = delegate.submit(taskAction);
|
||||||
|
final TaskAction actionKey;
|
||||||
|
if (taskAction instanceof SurrogateAction) {
|
||||||
|
actionKey = ((SurrogateAction) taskAction).getTaskAction();
|
||||||
|
} else {
|
||||||
|
actionKey = taskAction;
|
||||||
|
}
|
||||||
|
actionCountMap.computeIfAbsent(actionKey.getClass(), k -> new AtomicInteger()).incrementAndGet();
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getActionCount(Class<? extends TaskAction> actionClass)
|
||||||
|
{
|
||||||
|
final AtomicInteger count = actionCountMap.get(actionClass);
|
||||||
|
|
||||||
|
return count == null ? 0 : count.get();
|
||||||
|
}
|
||||||
|
}
|
|
@ -41,6 +41,8 @@ import org.apache.druid.indexer.TaskState;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
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.IngestionStatsAndErrorsTaskReportData;
|
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
import org.apache.druid.indexing.common.LockGranularity;
|
||||||
import org.apache.druid.indexing.common.TaskReport;
|
import org.apache.druid.indexing.common.TaskReport;
|
||||||
|
@ -459,7 +461,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createTuningConfigWithNumShards(1, null, true),
|
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 1, null), true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -501,7 +503,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createTuningConfigWithNumShards(2, ImmutableList.of("dim"), true),
|
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")), true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -558,7 +560,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAppendToExisting() throws Exception
|
public void testWriteNewSegmentsWithAppendToExistingWithLinearPartitioningSuccessfullyAppend() throws Exception
|
||||||
{
|
{
|
||||||
File tmpDir = temporaryFolder.newFolder();
|
File tmpDir = temporaryFolder.newFolder();
|
||||||
File tmpFile = File.createTempFile("druid", "index", tmpDir);
|
File tmpFile = File.createTempFile("druid", "index", tmpDir);
|
||||||
|
@ -787,7 +789,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
createTuningConfig(2, 2, null, 2L, null, null, false, true),
|
createTuningConfig(2, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -835,7 +837,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
createTuningConfig(3, 2, null, 2L, null, null, true, true),
|
createTuningConfig(3, 2, null, 2L, null, true, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -882,7 +884,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
createTuningConfig(3, 2, null, 2L, null, null, false, true),
|
createTuningConfig(3, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -950,7 +952,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
0
|
0
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
createTuningConfig(2, null, null, null, null, null, false, false), // ignore parse exception,
|
createTuningConfig(2, null, null, null, null, false, false), // ignore parse exception,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -998,7 +1000,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
0
|
0
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
createTuningConfig(2, null, null, null, null, null, false, true), // report parse exception
|
createTuningConfig(2, null, null, null, null, false, true), // report parse exception
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -1418,7 +1420,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
0
|
0
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
createTuningConfig(2, 1, null, null, null, null, true, true), // report parse exception
|
createTuningConfig(2, 1, null, null, null, true, true), // report parse exception
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -1483,7 +1485,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
0
|
0
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
createTuningConfig(2, null, null, null, null, null, false, true), // report parse exception
|
createTuningConfig(2, null, null, null, null, false, true), // report parse exception
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -1538,7 +1540,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
createTuningConfig(3, 2, null, 2L, null, null, false, true),
|
createTuningConfig(3, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -1607,7 +1609,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
createTuningConfig(3, 2, null, 2L, null, null, false, true),
|
createTuningConfig(3, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
null,
|
null,
|
||||||
|
@ -1634,6 +1636,35 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIndexTaskWitSingleDimPartitionsSpecThrowingException() throws Exception
|
||||||
|
{
|
||||||
|
final IndexTask task = new IndexTask(
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
createIngestionSpec(
|
||||||
|
useInputFormatApi,
|
||||||
|
jsonMapper,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
createTuningConfigWithPartitionsSpec(new SingleDimensionPartitionsSpec(null, 1, null, false), true),
|
||||||
|
false
|
||||||
|
),
|
||||||
|
null,
|
||||||
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
|
null,
|
||||||
|
rowIngestionMetersFactory,
|
||||||
|
appenderatorsManager
|
||||||
|
);
|
||||||
|
expectedException.expect(UnsupportedOperationException.class);
|
||||||
|
expectedException.expectMessage(
|
||||||
|
"partitionsSpec[org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec] is not supported"
|
||||||
|
);
|
||||||
|
task.isReady(createActionClient(task));
|
||||||
|
}
|
||||||
|
|
||||||
public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus status)
|
public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus status)
|
||||||
{
|
{
|
||||||
// full stacktrace will be too long and make tests brittle (e.g. if line # changes), just match the main message
|
// full stacktrace will be too long and make tests brittle (e.g. if line # changes), just match the main message
|
||||||
|
@ -1662,15 +1693,13 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
|
||||||
forceGuaranteedRollup,
|
forceGuaranteedRollup,
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static IndexTuningConfig createTuningConfigWithNumShards(
|
private static IndexTuningConfig createTuningConfigWithPartitionsSpec(
|
||||||
int numShards,
|
PartitionsSpec partitionsSpec,
|
||||||
@Nullable List<String> partitionDimensions,
|
|
||||||
boolean forceGuaranteedRollup
|
boolean forceGuaranteedRollup
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -1679,8 +1708,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
1,
|
1,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
numShards,
|
partitionsSpec,
|
||||||
partitionDimensions,
|
|
||||||
forceGuaranteedRollup,
|
forceGuaranteedRollup,
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -1691,8 +1719,7 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
@Nullable Integer maxRowsInMemory,
|
@Nullable Integer maxRowsInMemory,
|
||||||
@Nullable Long maxBytesInMemory,
|
@Nullable Long maxBytesInMemory,
|
||||||
@Nullable Long maxTotalRows,
|
@Nullable Long maxTotalRows,
|
||||||
@Nullable Integer numShards,
|
@Nullable PartitionsSpec partitionsSpec,
|
||||||
@Nullable List<String> partitionDimensions,
|
|
||||||
boolean forceGuaranteedRollup,
|
boolean forceGuaranteedRollup,
|
||||||
boolean reportParseException
|
boolean reportParseException
|
||||||
)
|
)
|
||||||
|
@ -1704,9 +1731,9 @@ public class IndexTaskTest extends IngestionTestBase
|
||||||
maxBytesInMemory,
|
maxBytesInMemory,
|
||||||
maxTotalRows,
|
maxTotalRows,
|
||||||
null,
|
null,
|
||||||
numShards,
|
|
||||||
partitionDimensions,
|
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
|
partitionsSpec,
|
||||||
INDEX_SPEC,
|
INDEX_SPEC,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -28,12 +28,10 @@ import org.apache.druid.indexing.common.SegmentLoaderFactory;
|
||||||
import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter;
|
import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.TestUtils;
|
import org.apache.druid.indexing.common.TestUtils;
|
||||||
import org.apache.druid.indexing.common.actions.LocalTaskActionClient;
|
|
||||||
import org.apache.druid.indexing.common.actions.SegmentInsertAction;
|
import org.apache.druid.indexing.common.actions.SegmentInsertAction;
|
||||||
import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
|
import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
|
||||||
import org.apache.druid.indexing.common.actions.TaskAction;
|
import org.apache.druid.indexing.common.actions.TaskAction;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
|
import org.apache.druid.indexing.common.actions.TaskActionToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.TaskAuditLogConfig;
|
|
||||||
import org.apache.druid.indexing.common.config.TaskStorageConfig;
|
import org.apache.druid.indexing.common.config.TaskStorageConfig;
|
||||||
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
|
||||||
import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
|
import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
|
||||||
|
@ -209,13 +207,13 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
|
||||||
return testUtils.getTestIndexMergerV9();
|
return testUtils.getTestIndexMergerV9();
|
||||||
}
|
}
|
||||||
|
|
||||||
public class TestLocalTaskActionClient extends LocalTaskActionClient
|
public class TestLocalTaskActionClient extends CountingLocalTaskActionClientForTest
|
||||||
{
|
{
|
||||||
private final Set<DataSegment> publishedSegments = new HashSet<>();
|
private final Set<DataSegment> publishedSegments = new HashSet<>();
|
||||||
|
|
||||||
private TestLocalTaskActionClient(Task task)
|
private TestLocalTaskActionClient(Task task)
|
||||||
{
|
{
|
||||||
super(task, taskStorage, createTaskActionToolbox(), new TaskAuditLogConfig(false));
|
super(task, taskStorage, createTaskActionToolbox());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -21,16 +21,18 @@ package org.apache.druid.indexing.common.task;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.data.input.InputRow;
|
import org.apache.druid.data.input.InputRow;
|
||||||
|
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.TaskLock;
|
import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.LockListAction;
|
import org.apache.druid.indexing.common.actions.LockListAction;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
|
import org.apache.druid.indexing.common.task.batch.partition.RangePartitionAnalysis;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.SegmentId;
|
import org.apache.druid.timeline.SegmentId;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
|
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -79,7 +81,8 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
|
||||||
INTERVAL_NORMAL, NORMAL_PARTITIONS
|
INTERVAL_NORMAL, NORMAL_PARTITIONS
|
||||||
);
|
);
|
||||||
|
|
||||||
private RangePartitionCachingLocalSegmentAllocator target;
|
private CachingSegmentAllocator target;
|
||||||
|
private SequenceNameFunction sequenceNameFunction;
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public ExpectedException exception = ExpectedException.none();
|
public ExpectedException exception = ExpectedException.none();
|
||||||
|
@ -93,14 +96,18 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
|
||||||
.map(RangePartitionCachingLocalSegmentAllocatorTest::createTaskLock)
|
.map(RangePartitionCachingLocalSegmentAllocatorTest::createTaskLock)
|
||||||
.collect(Collectors.toList())
|
.collect(Collectors.toList())
|
||||||
);
|
);
|
||||||
target = new RangePartitionCachingLocalSegmentAllocator(
|
final RangePartitionAnalysis partitionAnalysis = new RangePartitionAnalysis(
|
||||||
toolbox,
|
new SingleDimensionPartitionsSpec(null, 1, PARTITION_DIMENSION, false)
|
||||||
TASKID,
|
|
||||||
SUPERVISOR_TASKID,
|
|
||||||
DATASOURCE,
|
|
||||||
PARTITION_DIMENSION,
|
|
||||||
INTERVAL_TO_PARTITONS
|
|
||||||
);
|
);
|
||||||
|
INTERVAL_TO_PARTITONS.forEach(partitionAnalysis::updateBucket);
|
||||||
|
target = SegmentAllocators.forNonLinearPartitioning(
|
||||||
|
toolbox,
|
||||||
|
DATASOURCE,
|
||||||
|
TASKID,
|
||||||
|
new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID),
|
||||||
|
partitionAnalysis
|
||||||
|
);
|
||||||
|
sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -112,7 +119,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
|
||||||
exception.expect(IllegalStateException.class);
|
exception.expect(IllegalStateException.class);
|
||||||
exception.expectMessage("Failed to get shardSpec");
|
exception.expectMessage("Failed to get shardSpec");
|
||||||
|
|
||||||
String sequenceName = target.getSequenceName(interval, row);
|
String sequenceName = sequenceNameFunction.getSequenceName(interval, row);
|
||||||
allocate(row, sequenceName);
|
allocate(row, sequenceName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -148,7 +155,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
|
||||||
// getSequenceName_forIntervalAndRow_shouldUseISOFormatAndPartitionNumForRow
|
// getSequenceName_forIntervalAndRow_shouldUseISOFormatAndPartitionNumForRow
|
||||||
Interval interval = INTERVAL_NORMAL;
|
Interval interval = INTERVAL_NORMAL;
|
||||||
InputRow row = createInputRow(interval, PARTITION9);
|
InputRow row = createInputRow(interval, PARTITION9);
|
||||||
String sequenceName = target.getSequenceName(interval, row);
|
String sequenceName = sequenceNameFunction.getSequenceName(interval, row);
|
||||||
String expectedSequenceName = StringUtils.format("%s_%s_%d", TASKID, interval, 1);
|
String expectedSequenceName = StringUtils.format("%s_%s_%d", TASKID, interval, 1);
|
||||||
Assert.assertEquals(expectedSequenceName, sequenceName);
|
Assert.assertEquals(expectedSequenceName, sequenceName);
|
||||||
}
|
}
|
||||||
|
@ -189,7 +196,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
|
||||||
@Nullable String partitionEnd
|
@Nullable String partitionEnd
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
String sequenceName = target.getSequenceName(interval, row);
|
String sequenceName = sequenceNameFunction.getSequenceName(interval, row);
|
||||||
SegmentIdWithShardSpec segmentIdWithShardSpec = allocate(row, sequenceName);
|
SegmentIdWithShardSpec segmentIdWithShardSpec = allocate(row, sequenceName);
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
|
@ -209,7 +216,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
|
||||||
return target.allocate(row, sequenceName, null, false);
|
return target.allocate(row, sequenceName, null, false);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw new UncheckedIOException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,30 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess;
|
||||||
|
|
||||||
|
public class SupervisorTaskAccessWithNullClient extends SupervisorTaskAccess
|
||||||
|
{
|
||||||
|
public SupervisorTaskAccessWithNullClient(String supervisorTaskId)
|
||||||
|
{
|
||||||
|
super(supervisorTaskId, null);
|
||||||
|
}
|
||||||
|
}
|
|
@ -23,20 +23,23 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.data.input.InputRow;
|
import org.apache.druid.data.input.InputRow;
|
||||||
|
import org.apache.druid.data.input.MapBasedInputRow;
|
||||||
|
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.TaskLock;
|
import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.actions.LockListAction;
|
import org.apache.druid.indexing.common.actions.LockListAction;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
import org.apache.druid.indexing.common.task.HashPartitionCachingLocalSegmentAllocator;
|
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.indexing.common.task.NonLinearlyPartitionedSequenceNameFunction;
|
||||||
|
import org.apache.druid.indexing.common.task.SegmentAllocators;
|
||||||
|
import org.apache.druid.indexing.common.task.SequenceNameFunction;
|
||||||
|
import org.apache.druid.indexing.common.task.SupervisorTaskAccessWithNullClient;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.SegmentId;
|
import org.apache.druid.timeline.SegmentId;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
|
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -47,7 +50,6 @@ import java.io.IOException;
|
||||||
import java.io.UncheckedIOException;
|
import java.io.UncheckedIOException;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
public class HashPartitionCachingLocalSegmentAllocatorTest
|
public class HashPartitionCachingLocalSegmentAllocatorTest
|
||||||
{
|
{
|
||||||
|
@ -60,28 +62,30 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
|
||||||
private static final String DIMENSION = "dim";
|
private static final String DIMENSION = "dim";
|
||||||
private static final List<String> PARTITION_DIMENSIONS = ImmutableList.of(DIMENSION);
|
private static final List<String> PARTITION_DIMENSIONS = ImmutableList.of(DIMENSION);
|
||||||
private static final int NUM_PARTITONS = 1;
|
private static final int NUM_PARTITONS = 1;
|
||||||
private static final ShardSpecFactory SHARD_SPEC_FACTORY = new HashBasedNumberedShardSpecFactory(
|
|
||||||
PARTITION_DIMENSIONS,
|
|
||||||
NUM_PARTITONS
|
|
||||||
);
|
|
||||||
private static final int PARTITION_NUM = 0;
|
private static final int PARTITION_NUM = 0;
|
||||||
private static final Map<Interval, Pair<ShardSpecFactory, Integer>> ALLOCATE_SPEC = ImmutableMap.of(
|
private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec(
|
||||||
INTERVAL, Pair.of(SHARD_SPEC_FACTORY, NUM_PARTITONS)
|
null,
|
||||||
|
null,
|
||||||
|
Collections.singletonList(DIMENSION)
|
||||||
);
|
);
|
||||||
|
|
||||||
private HashPartitionCachingLocalSegmentAllocator target;
|
private CachingSegmentAllocator target;
|
||||||
|
private SequenceNameFunction sequenceNameFunction;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws IOException
|
public void setup() throws IOException
|
||||||
{
|
{
|
||||||
TaskToolbox toolbox = createToolbox();
|
TaskToolbox toolbox = createToolbox();
|
||||||
target = new HashPartitionCachingLocalSegmentAllocator(
|
HashPartitionAnalysis partitionAnalysis = new HashPartitionAnalysis(PARTITIONS_SPEC);
|
||||||
|
partitionAnalysis.updateBucket(INTERVAL, NUM_PARTITONS);
|
||||||
|
target = SegmentAllocators.forNonLinearPartitioning(
|
||||||
toolbox,
|
toolbox,
|
||||||
TASKID,
|
|
||||||
SUPERVISOR_TASKID,
|
|
||||||
DATASOURCE,
|
DATASOURCE,
|
||||||
ALLOCATE_SPEC
|
TASKID,
|
||||||
|
new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID),
|
||||||
|
partitionAnalysis
|
||||||
);
|
);
|
||||||
|
sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -89,7 +93,7 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
|
||||||
{
|
{
|
||||||
InputRow row = createInputRow();
|
InputRow row = createInputRow();
|
||||||
|
|
||||||
String sequenceName = target.getSequenceName(INTERVAL, row);
|
String sequenceName = sequenceNameFunction.getSequenceName(INTERVAL, row);
|
||||||
SegmentIdWithShardSpec segmentIdWithShardSpec = target.allocate(row, sequenceName, null, false);
|
SegmentIdWithShardSpec segmentIdWithShardSpec = target.allocate(row, sequenceName, null, false);
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
|
@ -102,13 +106,12 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
|
||||||
Assert.assertEquals(PARTITION_NUM, shardSpec.getPartitionNum());
|
Assert.assertEquals(PARTITION_NUM, shardSpec.getPartitionNum());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void getSequenceName()
|
public void getSequenceName()
|
||||||
{
|
{
|
||||||
// getSequenceName_forIntervalAndRow_shouldUseISOFormatAndPartitionNumForRow
|
// getSequenceName_forIntervalAndRow_shouldUseISOFormatAndPartitionNumForRow
|
||||||
InputRow row = createInputRow();
|
InputRow row = createInputRow();
|
||||||
String sequenceName = target.getSequenceName(INTERVAL, row);
|
String sequenceName = sequenceNameFunction.getSequenceName(INTERVAL, row);
|
||||||
String expectedSequenceName = StringUtils.format("%s_%s_%d", TASKID, INTERVAL, PARTITION_NUM);
|
String expectedSequenceName = StringUtils.format("%s_%s_%d", TASKID, INTERVAL, PARTITION_NUM);
|
||||||
Assert.assertEquals(expectedSequenceName, sequenceName);
|
Assert.assertEquals(expectedSequenceName, sequenceName);
|
||||||
}
|
}
|
||||||
|
@ -148,12 +151,11 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
|
||||||
|
|
||||||
private static InputRow createInputRow()
|
private static InputRow createInputRow()
|
||||||
{
|
{
|
||||||
long timestamp = INTERVAL.getStartMillis();
|
final long timestamp = INTERVAL.getStartMillis();
|
||||||
InputRow inputRow = EasyMock.mock(InputRow.class);
|
return new MapBasedInputRow(
|
||||||
EasyMock.expect(inputRow.getTimestamp()).andStubReturn(DateTimes.utc(timestamp));
|
timestamp,
|
||||||
EasyMock.expect(inputRow.getTimestampFromEpoch()).andStubReturn(timestamp);
|
Collections.singletonList(DIMENSION),
|
||||||
EasyMock.expect(inputRow.getDimension(DIMENSION)).andStubReturn(Collections.singletonList(DIMENSION));
|
ImmutableMap.of(DIMENSION, 1)
|
||||||
EasyMock.replay(inputRow);
|
);
|
||||||
return inputRow;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,9 +34,11 @@ import org.apache.druid.indexer.TaskLocation;
|
||||||
import org.apache.druid.indexer.TaskState;
|
import org.apache.druid.indexer.TaskState;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.TaskStatusPlus;
|
import org.apache.druid.indexer.TaskStatusPlus;
|
||||||
|
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.task.AbstractTask;
|
import org.apache.druid.indexing.common.task.AbstractTask;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
||||||
|
import org.apache.druid.indexing.common.task.SegmentAllocators;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.TestAppenderatorsManager;
|
import org.apache.druid.indexing.common.task.TestAppenderatorsManager;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
|
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
|
||||||
|
@ -685,8 +687,17 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd
|
||||||
|
|
||||||
// build LocalParallelIndexTaskClient
|
// build LocalParallelIndexTaskClient
|
||||||
final ParallelIndexSupervisorTaskClient taskClient = taskClientFactory.build(null, getId(), 0, null, 0);
|
final ParallelIndexSupervisorTaskClient taskClient = taskClientFactory.build(null, getId(), 0, null, 0);
|
||||||
|
final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) getIngestionSchema()
|
||||||
final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient);
|
.getTuningConfig()
|
||||||
|
.getGivenOrDefaultPartitionsSpec();
|
||||||
|
final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning(
|
||||||
|
toolbox,
|
||||||
|
new SupervisorTaskAccess(getSupervisorTaskId(), taskClient),
|
||||||
|
getIngestionSchema().getDataSchema(),
|
||||||
|
getTaskLockHelper(),
|
||||||
|
getIngestionSchema().getIOConfig().isAppendToExisting(),
|
||||||
|
partitionsSpec
|
||||||
|
);
|
||||||
|
|
||||||
final SegmentIdWithShardSpec segmentIdentifier = segmentAllocator.allocate(
|
final SegmentIdWithShardSpec segmentIdentifier = segmentAllocator.allocate(
|
||||||
new MapBasedInputRow(DateTimes.of("2017-01-01"), Collections.emptyList(), Collections.emptyMap()),
|
new MapBasedInputRow(DateTimes.of("2017-01-01"), Collections.emptyList(), Collections.emptyMap()),
|
||||||
|
|
|
@ -155,10 +155,8 @@ public class ParallelIndexSupervisorTaskSerdeTest
|
||||||
@Test
|
@Test
|
||||||
public void forceGuaranteedRollupWithSingleDimPartitionsMissingDimension()
|
public void forceGuaranteedRollupWithSingleDimPartitionsMissingDimension()
|
||||||
{
|
{
|
||||||
expectedException.expect(IllegalStateException.class);
|
expectedException.expect(IllegalArgumentException.class);
|
||||||
expectedException.expectMessage(
|
expectedException.expectMessage("partitionDimension must be specified");
|
||||||
"forceGuaranteedRollup is incompatible with partitionsSpec: partitionDimension must be specified"
|
|
||||||
);
|
|
||||||
|
|
||||||
new ParallelIndexSupervisorTaskBuilder()
|
new ParallelIndexSupervisorTaskBuilder()
|
||||||
.ingestionSpec(
|
.ingestionSpec(
|
||||||
|
|
|
@ -33,13 +33,13 @@ import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
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.StringDistribution;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch;
|
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
import org.apache.druid.testing.junit.LoggerCaptureRule;
|
import org.apache.druid.testing.junit.LoggerCaptureRule;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.apache.logging.log4j.core.LogEvent;
|
import org.apache.logging.log4j.core.LogEvent;
|
||||||
import org.easymock.Capture;
|
import org.easymock.Capture;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
|
|
|
@ -20,15 +20,23 @@
|
||||||
package org.apache.druid.indexing.common.task.batch.parallel;
|
package org.apache.druid.indexing.common.task.batch.parallel;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||||
|
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
|
import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
|
||||||
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
import org.hamcrest.Matchers;
|
import org.hamcrest.Matchers;
|
||||||
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
public class PartialHashSegmentGenerateTaskTest
|
public class PartialHashSegmentGenerateTaskTest
|
||||||
{
|
{
|
||||||
|
@ -71,4 +79,28 @@ public class PartialHashSegmentGenerateTaskTest
|
||||||
String id = target.getId();
|
String id = target.getId();
|
||||||
Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentGenerateTask.TYPE));
|
Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentGenerateTask.TYPE));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCreateHashPartitionAnalysisFromPartitionsSpecWithNumShardsReturningAnalysisOfValidNumBuckets()
|
||||||
|
{
|
||||||
|
final List<Interval> intervals = ImmutableList.of(
|
||||||
|
Intervals.of("2020-01-01/2020-01-02"),
|
||||||
|
Intervals.of("2020-01-02/2020-01-03"),
|
||||||
|
Intervals.of("2020-01-03/2020-01-04")
|
||||||
|
);
|
||||||
|
final int expectedNumBuckets = 5;
|
||||||
|
final HashPartitionAnalysis partitionAnalysis = PartialHashSegmentGenerateTask
|
||||||
|
.createHashPartitionAnalysisFromPartitionsSpec(
|
||||||
|
new UniformGranularitySpec(
|
||||||
|
Granularities.DAY,
|
||||||
|
Granularities.NONE,
|
||||||
|
intervals
|
||||||
|
),
|
||||||
|
new HashedPartitionsSpec(null, expectedNumBuckets, null)
|
||||||
|
);
|
||||||
|
Assert.assertEquals(intervals.size(), partitionAnalysis.getNumTimePartitions());
|
||||||
|
for (Interval interval : intervals) {
|
||||||
|
Assert.assertEquals(expectedNumBuckets, partitionAnalysis.getBucketAnalysis(interval).intValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,10 +28,10 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
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.java.util.common.Intervals;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.indexing.DataSchema;
|
import org.apache.druid.segment.indexing.DataSchema;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.hamcrest.Matchers;
|
import org.hamcrest.Matchers;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
|
|
|
@ -36,13 +36,13 @@ import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
|
||||||
import org.apache.druid.indexing.common.task.TaskResource;
|
import org.apache.druid.indexing.common.task.TaskResource;
|
||||||
import org.apache.druid.indexing.common.task.TestAppenderatorsManager;
|
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.ISE;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.guava.Comparators;
|
import org.apache.druid.java.util.common.guava.Comparators;
|
||||||
import org.apache.druid.query.scan.ScanResultValue;
|
import org.apache.druid.query.scan.ScanResultValue;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
|
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
|
||||||
import org.hamcrest.Matchers;
|
import org.hamcrest.Matchers;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.druid.indexing.common.task.batch.parallel.distribution;
|
package org.apache.druid.indexing.common.task.batch.parallel.distribution;
|
||||||
|
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.datasketches.quantiles.ItemsSketch;
|
||||||
import org.apache.druid.jackson.JacksonModule;
|
import org.apache.druid.jackson.JacksonModule;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
import org.apache.druid.timeline.partition.PartitionBoundaries;
|
||||||
import org.hamcrest.Matchers;
|
import org.hamcrest.Matchers;
|
||||||
import org.hamcrest.number.IsCloseTo;
|
import org.hamcrest.number.IsCloseTo;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
|
|
@ -54,13 +54,13 @@ import org.apache.druid.metadata.MetadataStorageTablesConfig;
|
||||||
import org.apache.druid.metadata.TestDerbyConnector;
|
import org.apache.druid.metadata.TestDerbyConnector;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory;
|
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.PartitionIds;
|
import org.apache.druid.timeline.partition.PartitionIds;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
@ -949,8 +949,8 @@ public class TaskLockboxTest
|
||||||
{
|
{
|
||||||
final Task task = NoopTask.create();
|
final Task task = NoopTask.create();
|
||||||
lockbox.add(task);
|
lockbox.add(task);
|
||||||
allocateSegmentsAndAssert(task, "seq", 3, NumberedShardSpecFactory.instance());
|
allocateSegmentsAndAssert(task, "seq", 3, NumberedPartialShardSpec.instance());
|
||||||
allocateSegmentsAndAssert(task, "seq2", 2, new NumberedOverwritingShardSpecFactory(0, 3, (short) 1));
|
allocateSegmentsAndAssert(task, "seq2", 2, new NumberedOverwritePartialShardSpec(0, 3, (short) 1));
|
||||||
|
|
||||||
final List<TaskLock> locks = lockbox.findLocksForTask(task);
|
final List<TaskLock> locks = lockbox.findLocksForTask(task);
|
||||||
Assert.assertEquals(5, locks.size());
|
Assert.assertEquals(5, locks.size());
|
||||||
|
@ -971,15 +971,15 @@ public class TaskLockboxTest
|
||||||
final Task task = NoopTask.create();
|
final Task task = NoopTask.create();
|
||||||
lockbox.add(task);
|
lockbox.add(task);
|
||||||
|
|
||||||
allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedShardSpecFactory(null, 3));
|
allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedPartialShardSpec(null, 3));
|
||||||
allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedShardSpecFactory(null, 5));
|
allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedPartialShardSpec(null, 5));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void allocateSegmentsAndAssert(
|
private void allocateSegmentsAndAssert(
|
||||||
Task task,
|
Task task,
|
||||||
String baseSequenceName,
|
String baseSequenceName,
|
||||||
int numSegmentsToAllocate,
|
int numSegmentsToAllocate,
|
||||||
ShardSpecFactory shardSpecFactory
|
PartialShardSpec partialShardSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
for (int i = 0; i < numSegmentsToAllocate; i++) {
|
for (int i = 0; i < numSegmentsToAllocate; i++) {
|
||||||
|
@ -988,7 +988,7 @@ public class TaskLockboxTest
|
||||||
TaskLockType.EXCLUSIVE,
|
TaskLockType.EXCLUSIVE,
|
||||||
task,
|
task,
|
||||||
Intervals.of("2015-01-01/2015-01-05"),
|
Intervals.of("2015-01-01/2015-01-05"),
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
StringUtils.format("%s_%d", baseSequenceName, i),
|
StringUtils.format("%s_%d", baseSequenceName, i),
|
||||||
null,
|
null,
|
||||||
true
|
true
|
||||||
|
@ -1013,7 +1013,7 @@ public class TaskLockboxTest
|
||||||
Assert.assertEquals(lockRequest.getGroupId(), segmentLock.getGroupId());
|
Assert.assertEquals(lockRequest.getGroupId(), segmentLock.getGroupId());
|
||||||
Assert.assertEquals(lockRequest.getDataSource(), segmentLock.getDataSource());
|
Assert.assertEquals(lockRequest.getDataSource(), segmentLock.getDataSource());
|
||||||
Assert.assertEquals(lockRequest.getInterval(), segmentLock.getInterval());
|
Assert.assertEquals(lockRequest.getInterval(), segmentLock.getInterval());
|
||||||
Assert.assertEquals(lockRequest.getShardSpecFactory().getShardSpecClass(), segmentId.getShardSpec().getClass());
|
Assert.assertEquals(lockRequest.getPartialShardSpec().getShardSpecClass(), segmentId.getShardSpec().getClass());
|
||||||
Assert.assertEquals(lockRequest.getPriority(), lockRequest.getPriority());
|
Assert.assertEquals(lockRequest.getPriority(), lockRequest.getPriority());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
import org.apache.druid.java.util.common.Pair;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -144,7 +144,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
|
||||||
String sequenceName,
|
String sequenceName,
|
||||||
String previousSegmentId,
|
String previousSegmentId,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
ShardSpecFactory shardSpecFactory,
|
PartialShardSpec partialShardSpec,
|
||||||
String maxVersion,
|
String maxVersion,
|
||||||
boolean skipSegmentLineageCheck
|
boolean skipSegmentLineageCheck
|
||||||
)
|
)
|
||||||
|
@ -153,7 +153,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
|
||||||
dataSource,
|
dataSource,
|
||||||
interval,
|
interval,
|
||||||
maxVersion,
|
maxVersion,
|
||||||
shardSpecFactory.create(objectMapper, 0)
|
partialShardSpec.complete(objectMapper, 0)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,7 +22,7 @@ package org.apache.druid.indexing.overlord;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
import org.apache.druid.java.util.common.Pair;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -157,7 +157,7 @@ public interface IndexerMetadataStorageCoordinator
|
||||||
* @param previousSegmentId previous segment in the series; may be null or empty, meaning this is the first
|
* @param previousSegmentId previous segment in the series; may be null or empty, meaning this is the first
|
||||||
* segment
|
* segment
|
||||||
* @param interval interval for which to allocate a segment
|
* @param interval interval for which to allocate a segment
|
||||||
* @param shardSpecFactory shardSpecFactory containing all necessary information to create a shardSpec for the
|
* @param partialShardSpec partialShardSpec containing all necessary information to create a shardSpec for the
|
||||||
* new segmentId
|
* new segmentId
|
||||||
* @param maxVersion use this version if we have no better version to use. The returned segment
|
* @param maxVersion use this version if we have no better version to use. The returned segment
|
||||||
* identifier may have a version lower than this one, but will not have one higher.
|
* identifier may have a version lower than this one, but will not have one higher.
|
||||||
|
@ -171,7 +171,7 @@ public interface IndexerMetadataStorageCoordinator
|
||||||
String sequenceName,
|
String sequenceName,
|
||||||
@Nullable String previousSegmentId,
|
@Nullable String previousSegmentId,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
ShardSpecFactory shardSpecFactory,
|
PartialShardSpec partialShardSpec,
|
||||||
String maxVersion,
|
String maxVersion,
|
||||||
boolean skipSegmentLineageCheck
|
boolean skipSegmentLineageCheck
|
||||||
);
|
);
|
||||||
|
|
|
@ -48,9 +48,9 @@ import org.apache.druid.timeline.Partitions;
|
||||||
import org.apache.druid.timeline.TimelineObjectHolder;
|
import org.apache.druid.timeline.TimelineObjectHolder;
|
||||||
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
||||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.PartitionChunk;
|
import org.apache.druid.timeline.partition.PartitionChunk;
|
||||||
import org.apache.druid.timeline.partition.ShardSpec;
|
import org.apache.druid.timeline.partition.ShardSpec;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.skife.jdbi.v2.Folder3;
|
import org.skife.jdbi.v2.Folder3;
|
||||||
import org.skife.jdbi.v2.Handle;
|
import org.skife.jdbi.v2.Handle;
|
||||||
|
@ -494,7 +494,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
final String sequenceName,
|
final String sequenceName,
|
||||||
@Nullable final String previousSegmentId,
|
@Nullable final String previousSegmentId,
|
||||||
final Interval interval,
|
final Interval interval,
|
||||||
final ShardSpecFactory shardSpecFactory,
|
final PartialShardSpec partialShardSpec,
|
||||||
final String maxVersion,
|
final String maxVersion,
|
||||||
final boolean skipSegmentLineageCheck
|
final boolean skipSegmentLineageCheck
|
||||||
)
|
)
|
||||||
|
@ -512,7 +512,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
dataSource,
|
dataSource,
|
||||||
sequenceName,
|
sequenceName,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
maxVersion
|
maxVersion
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
|
@ -522,7 +522,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
sequenceName,
|
sequenceName,
|
||||||
previousSegmentId,
|
previousSegmentId,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
maxVersion
|
maxVersion
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -537,7 +537,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
final String sequenceName,
|
final String sequenceName,
|
||||||
@Nullable final String previousSegmentId,
|
@Nullable final String previousSegmentId,
|
||||||
final Interval interval,
|
final Interval interval,
|
||||||
final ShardSpecFactory shardSpecFactory,
|
final PartialShardSpec partialShardSpec,
|
||||||
final String maxVersion
|
final String maxVersion
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -569,7 +569,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
handle,
|
handle,
|
||||||
dataSource,
|
dataSource,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
maxVersion
|
maxVersion
|
||||||
);
|
);
|
||||||
if (newIdentifier == null) {
|
if (newIdentifier == null) {
|
||||||
|
@ -611,7 +611,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
final String dataSource,
|
final String dataSource,
|
||||||
final String sequenceName,
|
final String sequenceName,
|
||||||
final Interval interval,
|
final Interval interval,
|
||||||
final ShardSpecFactory shardSpecFactory,
|
final PartialShardSpec partialShardSpec,
|
||||||
final String maxVersion
|
final String maxVersion
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -645,7 +645,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
handle,
|
handle,
|
||||||
dataSource,
|
dataSource,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
maxVersion
|
maxVersion
|
||||||
);
|
);
|
||||||
if (newIdentifier == null) {
|
if (newIdentifier == null) {
|
||||||
|
@ -788,7 +788,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
final Handle handle,
|
final Handle handle,
|
||||||
final String dataSource,
|
final String dataSource,
|
||||||
final Interval interval,
|
final Interval interval,
|
||||||
final ShardSpecFactory shardSpecFactory,
|
final PartialShardSpec partialShardSpec,
|
||||||
final String maxVersion
|
final String maxVersion
|
||||||
) throws IOException
|
) throws IOException
|
||||||
{
|
{
|
||||||
|
@ -812,8 +812,8 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
if (existingChunks
|
if (existingChunks
|
||||||
.stream()
|
.stream()
|
||||||
.flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false))
|
.flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false))
|
||||||
.anyMatch(chunk -> !chunk.getObject().getShardSpec().isCompatible(shardSpecFactory.getShardSpecClass()))) {
|
.anyMatch(chunk -> !chunk.getObject().getShardSpec().isCompatible(partialShardSpec.getShardSpecClass()))) {
|
||||||
// All existing segments should have a compatible shardSpec with shardSpecFactory.
|
// All existing segments should have a compatible shardSpec with partialShardSpec.
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -828,7 +828,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
// Here we check only the segments of the same shardSpec to find out the max partitionId.
|
// Here we check only the segments of the same shardSpec to find out the max partitionId.
|
||||||
// Note that OverwriteShardSpec has the higher range for partitionId than others.
|
// Note that OverwriteShardSpec has the higher range for partitionId than others.
|
||||||
// See PartitionIds.
|
// See PartitionIds.
|
||||||
.filter(chunk -> chunk.getObject().getShardSpec().getClass() == shardSpecFactory.getShardSpecClass())
|
.filter(chunk -> chunk.getObject().getShardSpec().getClass() == partialShardSpec.getShardSpecClass())
|
||||||
.max(Comparator.comparing(chunk -> chunk.getObject().getShardSpec().getPartitionNum()))
|
.max(Comparator.comparing(chunk -> chunk.getObject().getShardSpec().getPartitionNum()))
|
||||||
.map(chunk -> SegmentIdWithShardSpec.fromDataSegment(chunk.getObject()))
|
.map(chunk -> SegmentIdWithShardSpec.fromDataSegment(chunk.getObject()))
|
||||||
.orElse(null);
|
.orElse(null);
|
||||||
|
@ -845,7 +845,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
}
|
}
|
||||||
|
|
||||||
maxId = pendings.stream()
|
maxId = pendings.stream()
|
||||||
.filter(id -> id.getShardSpec().getClass() == shardSpecFactory.getShardSpecClass())
|
.filter(id -> id.getShardSpec().getClass() == partialShardSpec.getShardSpecClass())
|
||||||
.max((id1, id2) -> {
|
.max((id1, id2) -> {
|
||||||
final int versionCompare = id1.getVersion().compareTo(id2.getVersion());
|
final int versionCompare = id1.getVersion().compareTo(id2.getVersion());
|
||||||
if (versionCompare != 0) {
|
if (versionCompare != 0) {
|
||||||
|
@ -867,7 +867,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
}
|
}
|
||||||
|
|
||||||
if (maxId == null) {
|
if (maxId == null) {
|
||||||
final ShardSpec shardSpec = shardSpecFactory.create(jsonMapper, null);
|
final ShardSpec shardSpec = partialShardSpec.complete(jsonMapper, null);
|
||||||
String version = versionOfExistingChunks == null ? maxVersion : versionOfExistingChunks;
|
String version = versionOfExistingChunks == null ? maxVersion : versionOfExistingChunks;
|
||||||
return new SegmentIdWithShardSpec(dataSource, interval, version, shardSpec);
|
return new SegmentIdWithShardSpec(dataSource, interval, version, shardSpec);
|
||||||
} else if (!maxId.getInterval().equals(interval) || maxId.getVersion().compareTo(maxVersion) > 0) {
|
} else if (!maxId.getInterval().equals(interval) || maxId.getVersion().compareTo(maxVersion) > 0) {
|
||||||
|
@ -880,7 +880,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
);
|
);
|
||||||
return null;
|
return null;
|
||||||
} else {
|
} else {
|
||||||
final ShardSpec newShardSpec = shardSpecFactory.create(jsonMapper, maxId.getShardSpec());
|
final ShardSpec newShardSpec = partialShardSpec.complete(jsonMapper, maxId.getShardSpec());
|
||||||
return new SegmentIdWithShardSpec(
|
return new SegmentIdWithShardSpec(
|
||||||
dataSource,
|
dataSource,
|
||||||
maxId.getInterval(),
|
maxId.getInterval(),
|
||||||
|
|
|
@ -21,25 +21,32 @@ package org.apache.druid.segment.realtime.appenderator;
|
||||||
|
|
||||||
import org.apache.druid.data.input.InputRow;
|
import org.apache.druid.data.input.InputRow;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
public interface SegmentAllocator
|
public interface SegmentAllocator
|
||||||
{
|
{
|
||||||
/**
|
/**
|
||||||
* Allocates a new segment for a given timestamp.
|
* Allocates a new segment for a given timestamp. Even though its name is "allocate", this method is actually
|
||||||
|
* idempotent: given the same sequenceName, previousSegmentId, and skipSegmentLineageCheck, the implementation
|
||||||
|
* must return the same segment ID.
|
||||||
*
|
*
|
||||||
* @param row the event which triggered this allocation request
|
* @param row the event which triggered this allocation request
|
||||||
* @param sequenceName sequenceName for this allocation
|
* @param sequenceName sequenceName for this allocation
|
||||||
* @param previousSegmentId segment identifier returned on the previous call to allocate for your sequenceName
|
* @param previousSegmentId segment identifier returned on the previous call to allocate for your sequenceName.
|
||||||
|
* When skipSegmentLineageCheck is false, this can be null if it is the first call
|
||||||
|
* for the same sequenceName.
|
||||||
|
* When skipSegmentLineageCheck is true, this will be ignored.
|
||||||
* @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence.
|
* @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence.
|
||||||
* Should be set to false if replica tasks would index events in same order
|
* Should be set to false if replica tasks would index events in same order
|
||||||
*
|
*
|
||||||
* @return the pending segment identifier, or null if it was impossible to allocate a new segment
|
* @return the pending segment identifier, or null if it was impossible to allocate a new segment
|
||||||
*/
|
*/
|
||||||
|
@Nullable
|
||||||
SegmentIdWithShardSpec allocate(
|
SegmentIdWithShardSpec allocate(
|
||||||
InputRow row,
|
InputRow row,
|
||||||
String sequenceName,
|
String sequenceName,
|
||||||
String previousSegmentId,
|
@Nullable String previousSegmentId,
|
||||||
boolean skipSegmentLineageCheck
|
boolean skipSegmentLineageCheck
|
||||||
) throws IOException;
|
) throws IOException;
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,16 +34,16 @@ import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
|
|
||||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||||
|
import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec;
|
import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory;
|
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||||
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
|
import org.apache.druid.timeline.partition.PartialShardSpec;
|
||||||
import org.apache.druid.timeline.partition.PartitionIds;
|
import org.apache.druid.timeline.partition.PartitionIds;
|
||||||
import org.apache.druid.timeline.partition.ShardSpecFactory;
|
|
||||||
import org.assertj.core.api.Assertions;
|
import org.assertj.core.api.Assertions;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -853,7 +853,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
@Test
|
@Test
|
||||||
public void testAllocatePendingSegment()
|
public void testAllocatePendingSegment()
|
||||||
{
|
{
|
||||||
final ShardSpecFactory shardSpecFactory = NumberedShardSpecFactory.instance();
|
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
|
||||||
final String dataSource = "ds";
|
final String dataSource = "ds";
|
||||||
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
|
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
|
||||||
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment(
|
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment(
|
||||||
|
@ -861,7 +861,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
null,
|
null,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
"version",
|
"version",
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -873,7 +873,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
identifier.toString(),
|
identifier.toString(),
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
identifier.getVersion(),
|
identifier.getVersion(),
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -885,7 +885,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
identifier1.toString(),
|
identifier1.toString(),
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
identifier1.getVersion(),
|
identifier1.getVersion(),
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -897,7 +897,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
identifier1.toString(),
|
identifier1.toString(),
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
identifier1.getVersion(),
|
identifier1.getVersion(),
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -910,7 +910,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq1",
|
"seq1",
|
||||||
null,
|
null,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
"version",
|
"version",
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -921,7 +921,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
@Test
|
@Test
|
||||||
public void testDeletePendingSegment() throws InterruptedException
|
public void testDeletePendingSegment() throws InterruptedException
|
||||||
{
|
{
|
||||||
final ShardSpecFactory shardSpecFactory = NumberedShardSpecFactory.instance();
|
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
|
||||||
final String dataSource = "ds";
|
final String dataSource = "ds";
|
||||||
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
|
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
|
||||||
String prevSegmentId = null;
|
String prevSegmentId = null;
|
||||||
|
@ -934,7 +934,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
prevSegmentId,
|
prevSegmentId,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
"version",
|
"version",
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -949,7 +949,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
prevSegmentId,
|
prevSegmentId,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
"version",
|
"version",
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -973,7 +973,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
prevSegmentId,
|
prevSegmentId,
|
||||||
interval,
|
interval,
|
||||||
new NumberedOverwritingShardSpecFactory(0, 1, (short) (i + 1)),
|
new NumberedOverwritePartialShardSpec(0, 1, (short) (i + 1)),
|
||||||
"version",
|
"version",
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -1032,10 +1032,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
@Test
|
@Test
|
||||||
public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IOException
|
public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IOException
|
||||||
{
|
{
|
||||||
final ShardSpecFactory shardSpecFactory = new HashBasedNumberedShardSpecFactory(
|
final PartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(null, 5);
|
||||||
null,
|
|
||||||
5
|
|
||||||
);
|
|
||||||
final String dataSource = "ds";
|
final String dataSource = "ds";
|
||||||
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
|
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
|
||||||
|
|
||||||
|
@ -1044,7 +1041,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq",
|
"seq",
|
||||||
null,
|
null,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
"version",
|
"version",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -1074,7 +1071,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq2",
|
"seq2",
|
||||||
null,
|
null,
|
||||||
interval,
|
interval,
|
||||||
shardSpecFactory,
|
partialShardSpec,
|
||||||
"version",
|
"version",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -1104,7 +1101,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
"seq3",
|
"seq3",
|
||||||
null,
|
null,
|
||||||
interval,
|
interval,
|
||||||
new HashBasedNumberedShardSpecFactory(null, 3),
|
new HashBasedNumberedPartialShardSpec(null, 3),
|
||||||
"version",
|
"version",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
|
|
@ -36,6 +36,8 @@ import org.junit.Test;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
public class HashBasedNumberedShardSpecTest
|
public class HashBasedNumberedShardSpecTest
|
||||||
{
|
{
|
||||||
|
@ -126,7 +128,6 @@ public class HashBasedNumberedShardSpecTest
|
||||||
@Test
|
@Test
|
||||||
public void testIsInChunk()
|
public void testIsInChunk()
|
||||||
{
|
{
|
||||||
|
|
||||||
List<ShardSpec> specs = new ArrayList<>();
|
List<ShardSpec> specs = new ArrayList<>();
|
||||||
for (int i = 0; i < 3; i++) {
|
for (int i = 0; i < 3; i++) {
|
||||||
specs.add(new HashOverridenShardSpec(i, 3));
|
specs.add(new HashOverridenShardSpec(i, 3));
|
||||||
|
@ -137,7 +138,20 @@ public class HashBasedNumberedShardSpecTest
|
||||||
assertExistsInOneSpec(specs, new HashInputRow(0));
|
assertExistsInOneSpec(specs, new HashInputRow(0));
|
||||||
assertExistsInOneSpec(specs, new HashInputRow(1000));
|
assertExistsInOneSpec(specs, new HashInputRow(1000));
|
||||||
assertExistsInOneSpec(specs, new HashInputRow(-1000));
|
assertExistsInOneSpec(specs, new HashInputRow(-1000));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIsInChunkWithMorePartitionsBeyondNumBucketsReturningTrue()
|
||||||
|
{
|
||||||
|
final int numBuckets = 3;
|
||||||
|
final List<ShardSpec> specs = IntStream.range(0, 10)
|
||||||
|
.mapToObj(i -> new HashOverridenShardSpec(i, numBuckets))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
for (int i = 0; i < 10; i++) {
|
||||||
|
final InputRow row = new HashInputRow(numBuckets * 10000 + i);
|
||||||
|
Assert.assertTrue(specs.get(i).isInChunk(row.getTimestampFromEpoch(), row));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -186,10 +200,7 @@ public class HashBasedNumberedShardSpecTest
|
||||||
|
|
||||||
public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec
|
public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec
|
||||||
{
|
{
|
||||||
public HashOverridenShardSpec(
|
public HashOverridenShardSpec(int partitionNum, int partitions)
|
||||||
int partitionNum,
|
|
||||||
int partitions
|
|
||||||
)
|
|
||||||
{
|
{
|
||||||
super(partitionNum, partitions, null, ServerTestHelper.MAPPER);
|
super(partitionNum, partitions, null, ServerTestHelper.MAPPER);
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue