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:
Jihoon Son 2020-02-07 16:23:07 -08:00 committed by GitHub
parent b55657cc26
commit e81230f9ab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
82 changed files with 2233 additions and 1053 deletions

View File

@ -324,6 +324,11 @@
<artifactId>jetty-util</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>nl.jqno.equalsverifier</groupId>
<artifactId>equalsverifier</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId>

View File

@ -52,6 +52,12 @@ public class DynamicPartitionsSpec implements PartitionsSpec
this.maxTotalRows = maxTotalRows;
}
@Override
public SecondaryPartitionType getType()
{
return SecondaryPartitionType.LINEAR;
}
@Override
@JsonProperty
public Integer getMaxRowsPerSegment()

View File

@ -123,6 +123,12 @@ public class HashedPartitionsSpec implements DimensionBasedPartitionsSpec
return null;
}
@Override
public SecondaryPartitionType getType()
{
return SecondaryPartitionType.HASH;
}
@Nullable
@Override
@JsonProperty

View File

@ -41,6 +41,9 @@ public interface PartitionsSpec
String MAX_ROWS_PER_SEGMENT = "maxRowsPerSegment";
int HISTORICAL_NULL = -1;
@JsonIgnore
SecondaryPartitionType getType();
/**
* Returns the max number of rows per segment.
* Implementations can have different default values which it could be even null.
@ -55,28 +58,6 @@ public interface PartitionsSpec
*/
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.
*/
@ -101,4 +82,26 @@ public interface PartitionsSpec
*/
@JsonIgnore
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;
}
}

View File

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

View File

@ -138,6 +138,12 @@ public class SingleDimensionPartitionsSpec implements DimensionBasedPartitionsSp
return targetRowsPerSegment;
}
@Override
public SecondaryPartitionType getType()
{
return SecondaryPartitionType.RANGE;
}
@JsonIgnore
@Override
@NotNull

View File

@ -27,20 +27,22 @@ import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
public class HashBasedNumberedShardSpecFactory implements ShardSpecFactory
public class HashBasedNumberedPartialShardSpec implements PartialShardSpec
{
public static final String TYPE = "hashed";
@Nullable
private final List<String> partitionDimensions;
private final int numPartitions;
private final int numBuckets;
@JsonCreator
public HashBasedNumberedShardSpecFactory(
public HashBasedNumberedPartialShardSpec(
@JsonProperty("partitionDimensions") @Nullable List<String> partitionDimensions,
@JsonProperty("numPartitions") int numPartitions
@JsonProperty("numPartitions") int numBuckets
)
{
this.partitionDimensions = partitionDimensions;
this.numPartitions = numPartitions;
this.numBuckets = numBuckets;
}
@Nullable
@ -50,27 +52,28 @@ public class HashBasedNumberedShardSpecFactory implements ShardSpecFactory
return partitionDimensions;
}
@JsonProperty public int getNumPartitions()
@JsonProperty("numPartitions")
public int getNumBuckets()
{
return numPartitions;
return numBuckets;
}
@Override
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
{
final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId;
return new HashBasedNumberedShardSpec(
prevSpec == null ? 0 : prevSpec.getPartitionNum() + 1,
numPartitions,
numBuckets,
partitionDimensions,
objectMapper
);
}
@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
@ -88,14 +91,14 @@ public class HashBasedNumberedShardSpecFactory implements ShardSpecFactory
if (o == null || getClass() != o.getClass()) {
return false;
}
HashBasedNumberedShardSpecFactory that = (HashBasedNumberedShardSpecFactory) o;
return numPartitions == that.numPartitions &&
HashBasedNumberedPartialShardSpec that = (HashBasedNumberedPartialShardSpec) o;
return numBuckets == that.numBuckets &&
Objects.equals(partitionDimensions, that.partitionDimensions);
}
@Override
public int hashCode()
{
return Objects.hash(partitionDimensions, numPartitions);
return Objects.hash(partitionDimensions, numBuckets);
}
}

View File

@ -23,21 +23,21 @@ import com.fasterxml.jackson.databind.ObjectMapper;
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;
}
private LinearShardSpecFactory()
private LinearPartialShardSpec()
{
}
@Override
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
{
return new LinearShardSpec(
specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1
@ -45,7 +45,7 @@ public class LinearShardSpecFactory implements ShardSpecFactory
}
@Override
public ShardSpec create(ObjectMapper objectMapper, int partitionId)
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
{
return new LinearShardSpec(partitionId);
}

View File

@ -25,14 +25,14 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import javax.annotation.Nullable;
public class NumberedOverwritingShardSpecFactory implements ShardSpecFactory
public class NumberedOverwritePartialShardSpec implements PartialShardSpec
{
private final int startRootPartitionId;
private final int endRootPartitionId;
private final short minorVersion;
@JsonCreator
public NumberedOverwritingShardSpecFactory(
public NumberedOverwritePartialShardSpec(
@JsonProperty("startRootPartitionId") int startRootPartitionId,
@JsonProperty("endRootPartitionId") int endRootPartitionId,
@JsonProperty("minorVersion") short minorVersion
@ -62,7 +62,7 @@ public class NumberedOverwritingShardSpecFactory implements ShardSpecFactory
}
@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
// and could be null if all existing segments are first-generation segments.
@ -77,7 +77,7 @@ public class NumberedOverwritingShardSpecFactory implements ShardSpecFactory
}
@Override
public ShardSpec create(ObjectMapper objectMapper, int partitionId)
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
{
return new NumberedOverwriteShardSpec(partitionId, startRootPartitionId, endRootPartitionId, minorVersion);
}

View File

@ -23,21 +23,21 @@ import com.fasterxml.jackson.databind.ObjectMapper;
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;
}
private NumberedShardSpecFactory()
private NumberedPartialShardSpec()
{
}
@Override
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
{
if (specOfPreviousMaxPartitionId == null) {
return new NumberedShardSpec(0, 0);
@ -48,7 +48,7 @@ public class NumberedShardSpecFactory implements ShardSpecFactory
}
@Override
public ShardSpec create(ObjectMapper objectMapper, int partitionId)
public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
{
return new NumberedShardSpec(partitionId, 0);
}

View File

@ -19,38 +19,43 @@
package org.apache.druid.timeline.partition;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.databind.ObjectMapper;
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")
@JsonSubTypes({
@JsonSubTypes.Type(name = "numbered", value = NumberedShardSpecFactory.class),
@JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpecFactory.class),
@JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwritingShardSpecFactory.class),
@Type(name = "numbered", value = NumberedPartialShardSpec.class),
@Type(name = HashBasedNumberedPartialShardSpec.TYPE, value = HashBasedNumberedPartialShardSpec.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
* first call for the timeChunk where the new segment is created.
* Creates a new ShardSpec based on {@code specOfPreviousMaxPartitionId}. If it's null, it assumes that this is the
* 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
* 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();
}

View File

@ -17,7 +17,7 @@
* 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;
@ -25,6 +25,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
/**
@ -74,4 +75,31 @@ public class PartitionBoundaries extends ForwardingList<String> implements List<
{
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);
}
}

View File

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

View File

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

View File

@ -17,9 +17,11 @@
* 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.Before;
import org.junit.Test;
@ -74,8 +76,23 @@ public class PartitionBoundariesTest
}
@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();
}
}

View File

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

View File

@ -369,7 +369,14 @@ public class IndexGeneratorJobTest
new StringInputRowParser(
new JSONParseSpec(
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
),
@ -553,12 +560,22 @@ public class IndexGeneratorJobTest
List<ShardSpec> specs = new ArrayList<>();
if ("hashed".equals(partitionType)) {
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)) {
int partitionNum = 0;
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 {
throw new RE("Invalid partition type:[%s]", partitionType);

View File

@ -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.SegmentIdWithShardSpec;
import javax.annotation.Nullable;
import java.io.IOException;
public class ActionBasedSegmentAllocator implements SegmentAllocator
@ -44,6 +45,7 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
this.actionGenerator = actionGenerator;
}
@Nullable
@Override
public SegmentIdWithShardSpec allocate(
final InputRow row,

View File

@ -19,7 +19,6 @@
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.Task;
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.service.ServiceMetricEvent;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
public class LocalTaskActionClient implements TaskActionClient
{
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 TaskStorage storage;
private final TaskActionToolbox toolbox;
@ -79,17 +73,9 @@ public class LocalTaskActionClient implements TaskActionClient
final long performStartTime = System.currentTimeMillis();
final RetType result = taskAction.perform(task, toolbox);
emitTimerMetric("task/action/run/time", System.currentTimeMillis() - performStartTime);
actionCountMap.computeIfAbsent(taskAction.getClass(), k -> new AtomicInteger()).incrementAndGet();
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)
{
final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder();

View File

@ -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.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -63,6 +63,8 @@ import java.util.stream.Collectors;
*/
public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
{
public static final String TYPE = "segmentAllocate";
private static final Logger log = new Logger(SegmentAllocateAction.class);
// 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 previousSegmentId;
private final boolean skipSegmentLineageCheck;
private final ShardSpecFactory shardSpecFactory;
private final PartialShardSpec partialShardSpec;
private final LockGranularity lockGranularity;
@JsonCreator
@ -88,7 +90,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
@JsonProperty("previousSegmentId") String previousSegmentId,
@JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck,
// nullable for backward compatibility
@JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory,
@JsonProperty("shardSpecFactory") @Nullable PartialShardSpec partialShardSpec,
@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.previousSegmentId = previousSegmentId;
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;
}
@ -148,10 +150,10 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
return skipSegmentLineageCheck;
}
@JsonProperty
public ShardSpecFactory getShardSpecFactory()
@JsonProperty("shardSpecFactory")
public PartialShardSpec getPartialShardSpec()
{
return shardSpecFactory;
return partialShardSpec;
}
@JsonProperty
@ -293,7 +295,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
task.getGroupId(),
dataSource,
tryInterval,
shardSpecFactory,
partialShardSpec,
task.getPriority(),
sequenceName,
previousSegmentId,
@ -355,7 +357,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
", sequenceName='" + sequenceName + '\'' +
", previousSegmentId='" + previousSegmentId + '\'' +
", skipSegmentLineageCheck=" + skipSegmentLineageCheck +
", shardSpecFactory=" + shardSpecFactory +
", partialShardSpec=" + partialShardSpec +
", lockGranularity=" + lockGranularity +
'}';
}

View File

@ -27,8 +27,8 @@ import com.google.common.collect.ImmutableSet;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskLock;
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.TaskLockHelper;
import org.apache.druid.indexing.overlord.CriticalAction;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
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>> newSegmentsMap = groupSegmentsByIntervalAndSort(segments);
oldSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
newSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
oldSegmentsMap.values().forEach(TaskLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
newSegmentsMap.values().forEach(TaskLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> {
final List<DataSegment> newSegmentsPerInterval = Preconditions.checkNotNull(

View File

@ -40,7 +40,7 @@ import org.apache.druid.indexing.common.task.Task;
@JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class),
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.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 = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class),
@JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class)

View File

@ -24,8 +24,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskLock;
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.overlord.Segments;
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.GranularityType;
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.Partitions;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.function.Consumer;
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 final SegmentLockHelper segmentLockHelper;
@GuardedBy("this")
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")
private boolean stopped = false;
private TaskLockHelper taskLockHelper;
protected AbstractBatchIndexTask(String id, String dataSource, Map<String, Object> context)
{
super(id, dataSource, context);
segmentLockHelper = new SegmentLockHelper();
}
protected AbstractBatchIndexTask(
@ -107,7 +93,6 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
)
{
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);
}
public boolean isUseSegmentLock()
public TaskLockHelper getTaskLockHelper()
{
return useSegmentLock;
}
public SegmentLockHelper getSegmentLockHelper()
{
return segmentLockHelper;
return Preconditions.checkNotNull(taskLockHelper, "taskLockHelper is not initialized yet");
}
/**
@ -229,7 +209,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
// Respect task context value most.
if (forceTimeChunkLock) {
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()) {
return tryTimeChunkLock(client, intervals);
} else {
@ -238,7 +218,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
} else {
if (!intervals.isEmpty()) {
final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals);
useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT;
taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT);
return tryLockWithDetermineResult(client, result);
} else {
return true;
@ -255,14 +235,14 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
);
if (forceTimeChunkLock) {
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(
client,
new ArrayList<>(segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()))
);
} else {
final LockGranularityDetermineResult result = determineSegmentGranularity(segments);
useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT;
taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT);
return tryLockWithDetermineResult(client, result);
}
}
@ -298,7 +278,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
if (result.lockGranularity == LockGranularity.TIME_CHUNK) {
return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals"));
} else {
return segmentLockHelper.verifyAndLockExistingSegments(
return taskLockHelper.verifyAndLockExistingSegments(
client,
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
* 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
* 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();
}
static Pair<ShardSpecFactory, Integer> createShardSpecFactoryForGuaranteedRollup(
int numShards,
@Nullable List<String> partitionDimensions
)
{
return Pair.of(new HashBasedNumberedShardSpecFactory(partitionDimensions, numShards), numShards);
}
@Nullable
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
* from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be

View File

@ -90,7 +90,7 @@ import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory;
import org.apache.druid.segment.realtime.plumber.Committers;
import org.apache.druid.server.security.Action;
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 javax.servlet.http.HttpServletRequest;
@ -799,7 +799,7 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
sequenceName,
previousSegmentId,
skipSegmentLineageCheck,
NumberedShardSpecFactory.instance(),
NumberedPartialShardSpec.instance(),
LockGranularity.TIME_CHUNK
)
),

View File

@ -19,17 +19,21 @@
package org.apache.druid.indexing.common.task;
import com.google.common.base.Preconditions;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.LockListAction;
import org.apache.druid.indexing.common.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.batch.parallel.SupervisorTaskAccess;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.partition.ShardSpec;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
@ -41,11 +45,8 @@ import java.util.stream.Collectors;
/**
* 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 Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
@ -59,22 +60,34 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
*
* @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,
String dataSource,
String taskId,
String supervisorTaskId,
@Nullable SupervisorTaskAccess supervisorTaskAccess,
IntervalToSegmentIdsCreator intervalToSegmentIdsCreator
) throws IOException
{
this.taskId = taskId;
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 =
toolbox.getTaskActionClient()
.submit(new SurrogateAction<>(supervisorTaskId, new LockListAction()))
.submit(action)
.stream()
.collect(Collectors.toMap(
TaskLock::getInterval,
@ -82,7 +95,11 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
));
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<>();
for (Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) {
@ -115,14 +132,11 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
boolean skipSegmentLineageCheck
)
{
return sequenceNameToSegmentId.get(sequenceName);
}
@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));
return Preconditions.checkNotNull(
sequenceNameToSegmentId.get(sequenceName),
"Missing segmentId for the sequence[%s]",
sequenceName
);
}
/**
@ -136,4 +150,10 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
// and it's faster to add strings together than it is to use String#format
return taskId + "_" + interval + "_" + shardSpec.getPartitionNum();
}
@Override
public ShardSpecs getShardSpecs()
{
return shardSpecs;
}
}

View File

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

View File

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

View File

@ -53,9 +53,9 @@ import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SecondaryPartitionType;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
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.TaskReport;
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.RowIngestionMetersFactory;
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.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.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
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.UOE;
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.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.BaseAppenderatorDriver;
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.TransactionalSegmentPublisher;
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.NumberedShardSpec;
import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.apache.druid.utils.CircularBuffer;
import org.codehaus.plexus.util.FileUtils;
import org.joda.time.Interval;
import org.joda.time.Period;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.GET;
@ -130,6 +136,7 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
@ -264,6 +271,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@Override
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());
}
@ -487,13 +501,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
// Initialize maxRowsPerSegment and maxTotalRows lazily
final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig;
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = determineShardSpecs(
final PartitionAnalysis partitionAnalysis = determineShardSpecs(
toolbox,
inputSource,
tmpDir,
partitionsSpec
);
final List<Interval> allocateIntervals = new ArrayList<>(allocateSpec.keySet());
final List<Interval> allocateIntervals = new ArrayList<>(partitionAnalysis.getAllIntervalsToIndex());
final DataSchema dataSchema;
if (determineIntervals) {
if (!determineLockGranularityandTryLock(toolbox.getTaskActionClient(), allocateIntervals)) {
@ -508,14 +522,14 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
} else {
dataSchema = ingestionSchema.getDataSchema();
}
ingestionState = IngestionState.BUILD_SEGMENTS;
return generateAndPublishSegments(
toolbox,
dataSchema,
allocateSpec,
inputSource,
tmpDir,
partitionsSpec
partitionAnalysis
);
}
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.
*/
private Map<Interval, Pair<ShardSpecFactory, Integer>> determineShardSpecs(
private PartitionAnalysis determineShardSpecs(
final TaskToolbox toolbox,
final InputSource inputSource,
final File tmpDir,
final PartitionsSpec nonNullPartitionsSpec
@Nonnull final PartitionsSpec partitionsSpec
) throws IOException
{
final ObjectMapper jsonMapper = toolbox.getJsonMapper();
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final IndexIOConfig ioConfig = ingestionSchema.getIOConfig();
final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
@ -615,42 +627,59 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent();
// 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 (!determineNumPartitions && !determineIntervals) {
log.info("Skipping determine partition scan");
return createShardSpecWithoutInputScan(
granularitySpec,
ioConfig,
tuningConfig,
nonNullPartitionsSpec
);
if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
return PartialHashSegmentGenerateTask.createHashPartitionAnalysisFromPartitionsSpec(
granularitySpec,
(HashedPartitionsSpec) partitionsSpec
);
} else if (partitionsSpec.getType() == SecondaryPartitionType.LINEAR) {
return createLinearPartitionAnalysis(granularitySpec, (DynamicPartitionsSpec) partitionsSpec);
} else {
throw new UOE("%s", partitionsSpec.getClass().getName());
}
} else {
// determine intervals containing data and prime HLL collectors
log.info("Determining intervals and shardSpecs");
return createShardSpecsFromInput(
jsonMapper,
ingestionSchema,
inputSource,
tmpDir,
granularitySpec,
nonNullPartitionsSpec,
partitionsSpec,
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,
IndexIngestionSpec ingestionSchema,
InputSource inputSource,
File tmpDir,
GranularitySpec granularitySpec,
PartitionsSpec nonNullPartitionsSpec,
@Nonnull PartitionsSpec partitionsSpec,
boolean determineIntervals
) throws IOException
{
log.info("Determining intervals and shardSpecs");
assert partitionsSpec.getType() != SecondaryPartitionType.RANGE;
long determineShardSpecsStartMillis = System.currentTimeMillis();
final Map<Interval, Optional<HyperLogLogCollector>> hllCollectors = collectIntervalsAndShardSpecs(
@ -659,50 +688,49 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
inputSource,
tmpDir,
granularitySpec,
nonNullPartitionsSpec,
partitionsSpec,
determineIntervals
);
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpecs = new HashMap<>();
final PartitionAnalysis<Integer, ?> partitionAnalysis;
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()) {
final Interval interval = entry.getKey();
if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) {
assert nonNullPartitionsSpec instanceof HashedPartitionsSpec;
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) nonNullPartitionsSpec;
final int numBucketsPerInterval;
if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
final HashedPartitionsSpec hashedPartitionsSpec = (HashedPartitionsSpec) partitionsSpec;
final HyperLogLogCollector collector = entry.getValue().orNull();
final int numShards;
if (partitionsSpec.needsDeterminePartitions(false)) {
final long numRows = Preconditions.checkNotNull(collector, "HLL collector").estimateCardinalityRound();
final int nonNullMaxRowsPerSegment = partitionsSpec.getMaxRowsPerSegment() == null
? PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT
: partitionsSpec.getMaxRowsPerSegment();
numShards = (int) Math.ceil((double) numRows / nonNullMaxRowsPerSegment);
numBucketsPerInterval = (int) Math.ceil((double) numRows / nonNullMaxRowsPerSegment);
log.info(
"Estimated [%,d] rows of data for interval [%s], creating [%,d] shards",
numRows,
interval,
numShards
numBucketsPerInterval
);
} else {
numShards = partitionsSpec.getNumShards() == null ? 1 : partitionsSpec.getNumShards();
log.info("Creating [%,d] shards for interval [%s]", numShards, interval);
numBucketsPerInterval = hashedPartitionsSpec.getNumShards() == null ? 1 : hashedPartitionsSpec.getNumShards();
log.info("Creating [%,d] buckets for interval [%s]", numBucketsPerInterval, interval);
}
// Overwrite mode, guaranteed rollup: shardSpecs must be known in advance.
allocateSpecs.put(
interval,
createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions())
);
} else {
allocateSpecs.put(interval, null);
numBucketsPerInterval = 1;
}
partitionAnalysis.updateBucket(interval, numBucketsPerInterval);
}
log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis);
return allocateSpecs;
return partitionAnalysis;
}
private Map<Interval, Optional<HyperLogLogCollector>> collectIntervalsAndShardSpecs(
@ -711,7 +739,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
InputSource inputSource,
File tmpDir,
GranularitySpec granularitySpec,
PartitionsSpec nonNullPartitionsSpec,
@Nonnull PartitionsSpec partitionsSpec,
boolean determineIntervals
) throws IOException
{
@ -765,7 +793,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
interval = optInterval.get();
}
if (nonNullPartitionsSpec.needsDeterminePartitions(false)) {
if (partitionsSpec.needsDeterminePartitions(false)) {
hllCollectors.computeIfAbsent(interval, intv -> Optional.of(HyperLogLogCollector.makeLatestCollector()));
List<Object> groupKey = Rows.toGroupKey(
@ -809,32 +837,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
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}.
* 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(
final TaskToolbox toolbox,
final DataSchema dataSchema,
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec,
final InputSource inputSource,
final File tmpDir,
final PartitionsSpec partitionsSpec
final PartitionAnalysis partitionAnalysis
) throws IOException, InterruptedException
{
final FireDepartment fireDepartmentForMetrics =
@ -875,14 +876,42 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
}
final PartitionsSpec partitionsSpec = partitionAnalysis.getPartitionsSpec();
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final long pushTimeout = tuningConfig.getPushTimeout();
final IndexTaskSegmentAllocator segmentAllocator = createSegmentAllocator(
toolbox,
dataSchema,
allocateSpec
);
final SegmentAllocator segmentAllocator;
final SequenceNameFunction sequenceNameFunction;
switch (partitionsSpec.getType()) {
case HASH:
case RANGE:
final CachingSegmentAllocator localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning(
toolbox,
getDataSource(),
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) ->
toolbox.getTaskActionClient()
@ -921,13 +950,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
inputSource,
inputSource.needsFormat() ? getInputFormat(ingestionSchema) : null,
tmpDir,
segmentAllocator
sequenceNameFunction
);
// 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.
final Set<DataSegment> inputSegments = isUseSegmentLock()
? getSegmentLockHelper().getLockedExistingSegments()
final Set<DataSegment> inputSegments = getTaskLockHelper().isUseSegmentLock()
? getTaskLockHelper().getLockedExistingSegments()
: null;
// Probably we can publish atomicUpdateGroup along with segments.
final SegmentsAndCommitMetadata published =
@ -1207,11 +1236,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
static IndexTuningConfig createDefault()
{
return new IndexTuningConfig();
}
@Nullable
private static PartitionsSpec getDefaultPartitionsSpec(
boolean forceGuaranteedRollup,

View File

@ -96,7 +96,7 @@ public class InputSourceProcessor
InputSource inputSource,
@Nullable InputFormat inputFormat,
File tmpDir,
IndexTaskSegmentAllocator segmentAllocator
SequenceNameFunction sequenceNameFunction
) throws IOException, InterruptedException, ExecutionException, TimeoutException
{
@Nullable
@ -140,7 +140,7 @@ public class InputSourceProcessor
@SuppressWarnings("OptionalGetWithoutIsPresent")
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);
if (addResult.isOk()) {

View File

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

View File

@ -20,6 +20,8 @@
package org.apache.druid.indexing.common.task;
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.indexing.common.TaskLock;
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.Interval;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
/**
* 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;
LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec)
throws IOException
LocalSegmentAllocator(TaskToolbox toolbox, String dataSource, GranularitySpec granularitySpec) throws IOException
{
this.taskId = taskId;
final Map<Interval, AtomicInteger> counters = new HashMap<>();
final Map<Interval, String> intervalToVersion = toolbox.getTaskActionClient()
.submit(new LockListAction())
.stream()
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
final Map<Interval, String> intervalToVersion = toolbox
.getTaskActionClient()
.submit(new LockListAction())
.stream()
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
final Map<Interval, MutableInt> counters = Maps.newHashMapWithExpectedSize(intervalToVersion.size());
internalAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
final DateTime timestamp = row.getTimestamp();
@ -67,30 +64,25 @@ class LocalSegmentAllocator implements IndexTaskSegmentAllocator
}
final Interval interval = maybeInterval.get();
final String version = intervalToVersion.entrySet().stream()
.filter(entry -> entry.getKey().contains(interval))
.map(Entry::getValue)
.findFirst()
.orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
final String version = intervalToVersion
.entrySet()
.stream()
.filter(entry -> entry.getKey().contains(interval))
.map(Entry::getValue)
.findFirst()
.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(
dataSource,
interval,
version,
new NumberedShardSpec(partitionNum, 0)
new NumberedShardSpec(partitionId, 0)
);
};
}
@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;
}
@Nullable
@Override
public SegmentIdWithShardSpec allocate(
InputRow row,

View File

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

View File

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

View File

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

View File

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

View File

@ -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()
{
}
}

View File

@ -23,20 +23,13 @@ import org.apache.druid.data.input.InputRow;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
import org.joda.time.Interval;
/**
* Segment allocator interface for {@link IndexTask}. It has 3 different modes for allocating segments.
*/
public interface IndexTaskSegmentAllocator extends SegmentAllocator
public interface SequenceNameFunction
{
/**
* SequenceName is the key to create the segmentId. If previousSegmentId is given, {@link SegmentAllocator} allocates
* segmentId depending on sequenceName and previousSegmentId. If it's missing, it allocates segmentId using
* sequenceName and interval. For {@link IndexTask}, it always provides the previousSegmentId to
* SegmentAllocator.
* See {@link org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator#allocatePendingSegment} for details.
* The sequence name is to keep track of the row sequences that should be stored in the same segment.
* In other words, given two rows, they should be stored in the same segment if their sequence name is same.
*
* Implementations should return the correct sequenceName based on the given interval and inputRow, which is passed
* to SegmentAllocator.
* @see SegmentAllocator
*/
String getSequenceName(Interval interval, InputRow inputRow);
}

View File

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

View File

@ -20,6 +20,7 @@
package org.apache.druid.indexing.common.task;
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.TaskLockType;
import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction;
@ -46,12 +47,20 @@ import java.util.Set;
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 Set<DataSegment> lockedExistingSegments = new HashSet<>();
private final boolean useSegmentLock;
@Nullable
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()
{
return !lockedExistingSegments.isEmpty();

View File

@ -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.Tasks;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistributionMerger;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketchMerger;
@ -81,6 +80,7 @@ import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.apache.druid.utils.CollectionUtils;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import org.joda.time.DateTime;

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.druid.data.input.SplitHintSpec;
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.java.util.common.IAE;
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.maxNumSegmentsToMerge > 0, "maxNumSegmentsToMerge 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

View File

@ -26,22 +26,24 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.HashPartitionCachingLocalSegmentAllocator;
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
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.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.realtime.appenderator.AppenderatorsManager;
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 javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.SortedSet;
import java.util.stream.Collectors;
/**
@ -125,14 +127,18 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
}
@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,
getId(),
supervisorTaskId,
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 ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
return createShardSpecWithoutInputScan(
granularitySpec,
ingestionSchema.getIOConfig(),
tuningConfig,
partitionsSpec
);
final SortedSet<Interval> intervals = granularitySpec.bucketIntervals().get();
final int numBucketsPerInterval = partitionsSpec.getNumShards() == null
? 1
: partitionsSpec.getNumShards();
final HashPartitionAnalysis partitionAnalysis = new HashPartitionAnalysis(partitionsSpec);
intervals.forEach(interval -> partitionAnalysis.updateBucket(interval, numBucketsPerInterval));
return partitionAnalysis;
}
}

View File

@ -24,8 +24,8 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.joda.time.Interval;
import java.util.Map;

View File

@ -28,15 +28,16 @@ import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator;
import org.apache.druid.indexing.common.task.RangePartitionCachingLocalSegmentAllocator;
import org.apache.druid.indexing.common.task.SegmentAllocators;
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.partition.RangePartitionAnalysis;
import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -149,15 +150,19 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
}
@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,
getId(),
supervisorTaskId,
getDataSource(),
getPartitionDimension(ingestionSchema),
intervalToPartitions
getId(),
new SupervisorTaskAccess(supervisorTaskId, taskClient),
partitionAnalysis
);
}

View File

@ -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.RowIngestionMeters;
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.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator;
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.Tasks;
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.AppenderatorsManager;
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.timeline.DataSegment;
@ -117,16 +120,19 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
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));
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.
@ -138,6 +144,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
private List<DataSegment> generateSegments(
final TaskToolbox toolbox,
final ParallelIndexSupervisorTaskClient taskClient,
final InputSource inputSource,
final File tmpDir
) throws IOException, InterruptedException, ExecutionException, TimeoutException
@ -164,7 +171,11 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
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(
getId(),
@ -195,7 +206,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
inputSource,
inputSource.needsFormat() ? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema) : null,
tmpDir,
segmentAllocator
sequenceNameFunction
);
return pushed.getSegments();

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import org.apache.commons.io.FileUtils;
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.indexer.TaskStatus;
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.actions.SegmentAllocateAction;
import org.apache.druid.indexing.common.actions.SurrogateAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.AbstractBatchIndexTask;
import org.apache.druid.indexing.common.task.BatchAppenderators;
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.SegmentLockHelper;
import org.apache.druid.indexing.common.task.SegmentLockHelper.OverwritingRootGenerationPartitions;
import org.apache.druid.indexing.common.task.SegmentAllocators;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.Tasks;
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.BatchAppenderatorDriver;
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.timeline.DataSegment;
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.ShardSpecFactory;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -230,7 +220,7 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
);
// Find inputSegments overshadowed by pushedSegments
final Set<DataSegment> allSegments = new HashSet<>(getSegmentLockHelper().getLockedExistingSegments());
final Set<DataSegment> allSegments = new HashSet<>(getTaskLockHelper().getLockedExistingSegments());
allSegments.addAll(pushedSegments);
final VersionedIntervalTimeline<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(allSegments);
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}.
* 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 long pushTimeout = tuningConfig.getPushTimeout();
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(
getId(),

View File

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

View File

@ -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.JsonTypeInfo;
import org.apache.druid.timeline.partition.PartitionBoundaries;
/**
* Counts frequencies of {@link String}s.

View File

@ -34,6 +34,7 @@ import com.google.common.base.Preconditions;
import org.apache.datasketches.ArrayOfStringsSerDe;
import org.apache.datasketches.memory.Memory;
import org.apache.datasketches.quantiles.ItemsSketch;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import java.io.IOException;
import java.util.Comparator;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -25,7 +25,7 @@ import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
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 javax.annotation.Nullable;
@ -37,7 +37,7 @@ public class LockRequestForNewSegment implements LockRequest
private final String groupId;
private final String dataSource;
private final Interval interval;
private final ShardSpecFactory shardSpecFactory;
private final PartialShardSpec partialShardSpec;
private final int priority;
private final String sequenceName;
@Nullable
@ -52,7 +52,7 @@ public class LockRequestForNewSegment implements LockRequest
String groupId,
String dataSource,
Interval interval,
ShardSpecFactory shardSpecFactory,
PartialShardSpec partialShardSpec,
int priority,
String sequenceName,
@Nullable String previsousSegmentId,
@ -64,7 +64,7 @@ public class LockRequestForNewSegment implements LockRequest
this.groupId = groupId;
this.dataSource = dataSource;
this.interval = interval;
this.shardSpecFactory = shardSpecFactory;
this.partialShardSpec = partialShardSpec;
this.priority = priority;
this.sequenceName = sequenceName;
this.previsousSegmentId = previsousSegmentId;
@ -77,7 +77,7 @@ public class LockRequestForNewSegment implements LockRequest
TaskLockType lockType,
Task task,
Interval interval,
ShardSpecFactory shardSpecFactory,
PartialShardSpec partialShardSpec,
String sequenceName,
@Nullable String previsousSegmentId,
boolean skipSegmentLineageCheck
@ -89,7 +89,7 @@ public class LockRequestForNewSegment implements LockRequest
task.getGroupId(),
task.getDataSource(),
interval,
shardSpecFactory,
partialShardSpec,
task.getPriority(),
sequenceName,
previsousSegmentId,
@ -133,9 +133,9 @@ public class LockRequestForNewSegment implements LockRequest
return priority;
}
public ShardSpecFactory getShardSpecFactory()
public PartialShardSpec getPartialShardSpec()
{
return shardSpecFactory;
return partialShardSpec;
}
@Override
@ -187,7 +187,7 @@ public class LockRequestForNewSegment implements LockRequest
", groupId='" + groupId + '\'' +
", dataSource='" + dataSource + '\'' +
", interval=" + interval +
", shardSpecFactory=" + shardSpecFactory +
", partialShardSpec=" + partialShardSpec +
", priority=" + priority +
", sequenceName='" + sequenceName + '\'' +
", previsousSegmentId='" + previsousSegmentId + '\'' +

View File

@ -534,7 +534,7 @@ public class TaskLockbox
request.getSequenceName(),
request.getPrevisousSegmentId(),
request.getInterval(),
request.getShardSpecFactory(),
request.getPartialShardSpec(),
version,
request.isSkipSegmentLineageCheck()
);

View File

@ -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.ChatHandlerProvider;
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 javax.annotation.Nullable;
@ -232,7 +232,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
sequenceName,
previousSegmentId,
skipSegmentLineageCheck,
NumberedShardSpecFactory.instance(),
NumberedPartialShardSpec.instance(),
lockGranularityToUse
)
),

View File

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

View File

@ -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.segment.realtime.appenderator.SegmentIdWithShardSpec;
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.HashBasedNumberedShardSpecFactory;
import org.apache.druid.timeline.partition.LinearPartialShardSpec;
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.NumberedShardSpecFactory;
import org.apache.druid.timeline.partition.PartialShardSpec;
import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
@ -652,7 +652,7 @@ public class SegmentAllocateActionTest
Granularities.HOUR,
"s1",
null,
LinearShardSpecFactory.instance()
LinearPartialShardSpec.instance()
);
final SegmentIdWithShardSpec id2 = allocate(
task,
@ -661,7 +661,7 @@ public class SegmentAllocateActionTest
Granularities.HOUR,
"s1",
id1.toString(),
LinearShardSpecFactory.instance()
LinearPartialShardSpec.instance()
);
assertSameIdentifier(
@ -899,39 +899,7 @@ public class SegmentAllocateActionTest
}
@Test
public void testSerde() throws Exception
{
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
public void testWithPartialShardSpecAndOvershadowingSegments() throws IOException
{
final Task task = NoopTask.create();
taskActionTestKit.getTaskLockbox().add(task);
@ -965,7 +933,7 @@ public class SegmentAllocateActionTest
"seq",
null,
true,
new HashBasedNumberedShardSpecFactory(ImmutableList.of("dim1"), 2),
new HashBasedNumberedPartialShardSpec(ImmutableList.of("dim1"), 2),
lockGranularity
);
final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox());
@ -996,7 +964,7 @@ public class SegmentAllocateActionTest
preferredSegmentGranularity,
sequenceName,
sequencePreviousId,
NumberedShardSpecFactory.instance()
NumberedPartialShardSpec.instance()
);
}
@ -1007,7 +975,7 @@ public class SegmentAllocateActionTest
final Granularity preferredSegmentGranularity,
final String sequenceName,
final String sequencePreviousId,
final ShardSpecFactory shardSpecFactory
final PartialShardSpec partialShardSpec
)
{
final SegmentAllocateAction action = new SegmentAllocateAction(
@ -1018,7 +986,7 @@ public class SegmentAllocateActionTest
sequenceName,
sequencePreviousId,
false,
shardSpecFactory,
partialShardSpec,
lockGranularity
);
return action.perform(task, taskActionTestKit.getTaskActionToolbox());

View File

@ -236,7 +236,7 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
Granularities.MINUTE,
null
),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
false
),
null,

View File

@ -386,7 +386,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
Granularities.MINUTE,
null
),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
false
),
null,
@ -726,7 +726,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
),
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,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
@ -796,7 +796,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
Granularities.MINUTE,
null
),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
appendToExisting
),
null,

View File

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

View File

@ -41,6 +41,8 @@ import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskReport;
@ -459,7 +461,7 @@ public class IndexTaskTest extends IngestionTestBase
null,
null,
null,
createTuningConfigWithNumShards(1, null, true),
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 1, null), true),
false
),
null,
@ -501,7 +503,7 @@ public class IndexTaskTest extends IngestionTestBase
null,
null,
null,
createTuningConfigWithNumShards(2, ImmutableList.of("dim"), true),
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")), true),
false
),
null,
@ -558,7 +560,7 @@ public class IndexTaskTest extends IngestionTestBase
}
@Test
public void testAppendToExisting() throws Exception
public void testWriteNewSegmentsWithAppendToExistingWithLinearPartitioningSuccessfullyAppend() throws Exception
{
File tmpDir = temporaryFolder.newFolder();
File tmpFile = File.createTempFile("druid", "index", tmpDir);
@ -787,7 +789,7 @@ public class IndexTaskTest extends IngestionTestBase
Granularities.MINUTE,
null
),
createTuningConfig(2, 2, null, 2L, null, null, false, true),
createTuningConfig(2, 2, null, 2L, null, false, true),
false
),
null,
@ -835,7 +837,7 @@ public class IndexTaskTest extends IngestionTestBase
true,
null
),
createTuningConfig(3, 2, null, 2L, null, null, true, true),
createTuningConfig(3, 2, null, 2L, null, true, true),
false
),
null,
@ -882,7 +884,7 @@ public class IndexTaskTest extends IngestionTestBase
true,
null
),
createTuningConfig(3, 2, null, 2L, null, null, false, true),
createTuningConfig(3, 2, null, 2L, null, false, true),
false
),
null,
@ -950,7 +952,7 @@ public class IndexTaskTest extends IngestionTestBase
0
),
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
);
@ -998,7 +1000,7 @@ public class IndexTaskTest extends IngestionTestBase
0
),
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
);
@ -1418,7 +1420,7 @@ public class IndexTaskTest extends IngestionTestBase
0
),
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
);
@ -1483,7 +1485,7 @@ public class IndexTaskTest extends IngestionTestBase
0
),
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
);
@ -1538,7 +1540,7 @@ public class IndexTaskTest extends IngestionTestBase
true,
null
),
createTuningConfig(3, 2, null, 2L, null, null, false, true),
createTuningConfig(3, 2, null, 2L, null, false, true),
false
),
null,
@ -1607,7 +1609,7 @@ public class IndexTaskTest extends IngestionTestBase
true,
null
),
createTuningConfig(3, 2, null, 2L, null, null, false, true),
createTuningConfig(3, 2, null, 2L, null, false, true),
false
),
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)
{
// 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,
forceGuaranteedRollup,
true
);
}
private static IndexTuningConfig createTuningConfigWithNumShards(
int numShards,
@Nullable List<String> partitionDimensions,
private static IndexTuningConfig createTuningConfigWithPartitionsSpec(
PartitionsSpec partitionsSpec,
boolean forceGuaranteedRollup
)
{
@ -1679,8 +1708,7 @@ public class IndexTaskTest extends IngestionTestBase
1,
null,
null,
numShards,
partitionDimensions,
partitionsSpec,
forceGuaranteedRollup,
true
);
@ -1691,8 +1719,7 @@ public class IndexTaskTest extends IngestionTestBase
@Nullable Integer maxRowsInMemory,
@Nullable Long maxBytesInMemory,
@Nullable Long maxTotalRows,
@Nullable Integer numShards,
@Nullable List<String> partitionDimensions,
@Nullable PartitionsSpec partitionsSpec,
boolean forceGuaranteedRollup,
boolean reportParseException
)
@ -1704,9 +1731,9 @@ public class IndexTaskTest extends IngestionTestBase
maxBytesInMemory,
maxTotalRows,
null,
numShards,
partitionDimensions,
null,
null,
partitionsSpec,
INDEX_SPEC,
null,
null,

View File

@ -28,12 +28,10 @@ import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter;
import org.apache.druid.indexing.common.TaskToolbox;
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.SegmentTransactionalInsertAction;
import org.apache.druid.indexing.common.actions.TaskAction;
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.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
@ -209,13 +207,13 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
return testUtils.getTestIndexMergerV9();
}
public class TestLocalTaskActionClient extends LocalTaskActionClient
public class TestLocalTaskActionClient extends CountingLocalTaskActionClientForTest
{
private final Set<DataSegment> publishedSegments = new HashSet<>();
private TestLocalTaskActionClient(Task task)
{
super(task, taskStorage, createTaskActionToolbox(), new TaskAuditLogConfig(false));
super(task, taskStorage, createTaskActionToolbox());
}
@Override

View File

@ -21,16 +21,18 @@ package org.apache.druid.indexing.common.task;
import com.google.common.collect.ImmutableMap;
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.TaskToolbox;
import org.apache.druid.indexing.common.actions.LockListAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.indexing.common.task.batch.partition.RangePartitionAnalysis;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.easymock.EasyMock;
import org.joda.time.Interval;
@ -79,7 +81,8 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
INTERVAL_NORMAL, NORMAL_PARTITIONS
);
private RangePartitionCachingLocalSegmentAllocator target;
private CachingSegmentAllocator target;
private SequenceNameFunction sequenceNameFunction;
@Rule
public ExpectedException exception = ExpectedException.none();
@ -93,14 +96,18 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
.map(RangePartitionCachingLocalSegmentAllocatorTest::createTaskLock)
.collect(Collectors.toList())
);
target = new RangePartitionCachingLocalSegmentAllocator(
toolbox,
TASKID,
SUPERVISOR_TASKID,
DATASOURCE,
PARTITION_DIMENSION,
INTERVAL_TO_PARTITONS
final RangePartitionAnalysis partitionAnalysis = new RangePartitionAnalysis(
new SingleDimensionPartitionsSpec(null, 1, PARTITION_DIMENSION, false)
);
INTERVAL_TO_PARTITONS.forEach(partitionAnalysis::updateBucket);
target = SegmentAllocators.forNonLinearPartitioning(
toolbox,
DATASOURCE,
TASKID,
new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID),
partitionAnalysis
);
sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs());
}
@Test
@ -112,7 +119,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
exception.expect(IllegalStateException.class);
exception.expectMessage("Failed to get shardSpec");
String sequenceName = target.getSequenceName(interval, row);
String sequenceName = sequenceNameFunction.getSequenceName(interval, row);
allocate(row, sequenceName);
}
@ -148,7 +155,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
// getSequenceName_forIntervalAndRow_shouldUseISOFormatAndPartitionNumForRow
Interval interval = INTERVAL_NORMAL;
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);
Assert.assertEquals(expectedSequenceName, sequenceName);
}
@ -189,7 +196,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
@Nullable String partitionEnd
)
{
String sequenceName = target.getSequenceName(interval, row);
String sequenceName = sequenceNameFunction.getSequenceName(interval, row);
SegmentIdWithShardSpec segmentIdWithShardSpec = allocate(row, sequenceName);
Assert.assertEquals(
@ -209,7 +216,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest
return target.allocate(row, sequenceName, null, false);
}
catch (IOException e) {
throw new UncheckedIOException(e);
throw new RuntimeException(e);
}
}

View File

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

View File

@ -23,20 +23,23 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.LockListAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.HashPartitionCachingLocalSegmentAllocator;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
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.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.Assert;
@ -47,7 +50,6 @@ import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public class HashPartitionCachingLocalSegmentAllocatorTest
{
@ -60,28 +62,30 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
private static final String DIMENSION = "dim";
private static final List<String> PARTITION_DIMENSIONS = ImmutableList.of(DIMENSION);
private static final int NUM_PARTITONS = 1;
private static final ShardSpecFactory SHARD_SPEC_FACTORY = new HashBasedNumberedShardSpecFactory(
PARTITION_DIMENSIONS,
NUM_PARTITONS
);
private static final int PARTITION_NUM = 0;
private static final Map<Interval, Pair<ShardSpecFactory, Integer>> ALLOCATE_SPEC = ImmutableMap.of(
INTERVAL, Pair.of(SHARD_SPEC_FACTORY, NUM_PARTITONS)
private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec(
null,
null,
Collections.singletonList(DIMENSION)
);
private HashPartitionCachingLocalSegmentAllocator target;
private CachingSegmentAllocator target;
private SequenceNameFunction sequenceNameFunction;
@Before
public void setup() throws IOException
{
TaskToolbox toolbox = createToolbox();
target = new HashPartitionCachingLocalSegmentAllocator(
HashPartitionAnalysis partitionAnalysis = new HashPartitionAnalysis(PARTITIONS_SPEC);
partitionAnalysis.updateBucket(INTERVAL, NUM_PARTITONS);
target = SegmentAllocators.forNonLinearPartitioning(
toolbox,
TASKID,
SUPERVISOR_TASKID,
DATASOURCE,
ALLOCATE_SPEC
TASKID,
new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID),
partitionAnalysis
);
sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs());
}
@Test
@ -89,7 +93,7 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
{
InputRow row = createInputRow();
String sequenceName = target.getSequenceName(INTERVAL, row);
String sequenceName = sequenceNameFunction.getSequenceName(INTERVAL, row);
SegmentIdWithShardSpec segmentIdWithShardSpec = target.allocate(row, sequenceName, null, false);
Assert.assertEquals(
@ -102,13 +106,12 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
Assert.assertEquals(PARTITION_NUM, shardSpec.getPartitionNum());
}
@Test
public void getSequenceName()
{
// getSequenceName_forIntervalAndRow_shouldUseISOFormatAndPartitionNumForRow
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);
Assert.assertEquals(expectedSequenceName, sequenceName);
}
@ -148,12 +151,11 @@ public class HashPartitionCachingLocalSegmentAllocatorTest
private static InputRow createInputRow()
{
long timestamp = INTERVAL.getStartMillis();
InputRow inputRow = EasyMock.mock(InputRow.class);
EasyMock.expect(inputRow.getTimestamp()).andStubReturn(DateTimes.utc(timestamp));
EasyMock.expect(inputRow.getTimestampFromEpoch()).andStubReturn(timestamp);
EasyMock.expect(inputRow.getDimension(DIMENSION)).andStubReturn(Collections.singletonList(DIMENSION));
EasyMock.replay(inputRow);
return inputRow;
final long timestamp = INTERVAL.getStartMillis();
return new MapBasedInputRow(
timestamp,
Collections.singletonList(DIMENSION),
ImmutableMap.of(DIMENSION, 1)
);
}
}

View File

@ -34,9 +34,11 @@ import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
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.task.AbstractTask;
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.TestAppenderatorsManager;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
@ -685,8 +687,17 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd
// build LocalParallelIndexTaskClient
final ParallelIndexSupervisorTaskClient taskClient = taskClientFactory.build(null, getId(), 0, null, 0);
final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient);
final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) getIngestionSchema()
.getTuningConfig()
.getGivenOrDefaultPartitionsSpec();
final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning(
toolbox,
new SupervisorTaskAccess(getSupervisorTaskId(), taskClient),
getIngestionSchema().getDataSchema(),
getTaskLockHelper(),
getIngestionSchema().getIOConfig().isAppendToExisting(),
partitionsSpec
);
final SegmentIdWithShardSpec segmentIdentifier = segmentAllocator.allocate(
new MapBasedInputRow(DateTimes.of("2017-01-01"), Collections.emptyList(), Collections.emptyMap()),

View File

@ -155,10 +155,8 @@ public class ParallelIndexSupervisorTaskSerdeTest
@Test
public void forceGuaranteedRollupWithSingleDimPartitionsMissingDimension()
{
expectedException.expect(IllegalStateException.class);
expectedException.expectMessage(
"forceGuaranteedRollup is incompatible with partitionsSpec: partitionDimension must be specified"
);
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("partitionDimension must be specified");
new ParallelIndexSupervisorTaskBuilder()
.ingestionSpec(

View File

@ -33,13 +33,13 @@ import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.testing.junit.LoggerCaptureRule;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.apache.logging.log4j.core.LogEvent;
import org.easymock.Capture;
import org.easymock.EasyMock;

View File

@ -20,15 +20,23 @@
package org.apache.druid.indexing.common.task.batch.parallel;
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.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.indexing.granularity.UniformGranularitySpec;
import org.hamcrest.Matchers;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.File;
import java.util.List;
public class PartialHashSegmentGenerateTaskTest
{
@ -71,4 +79,28 @@ public class PartialHashSegmentGenerateTaskTest
String id = target.getId();
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());
}
}
}

View File

@ -28,10 +28,10 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.hamcrest.Matchers;
import org.junit.Assert;
import org.junit.Rule;

View File

@ -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.TaskResource;
import org.apache.druid.indexing.common.task.TestAppenderatorsManager;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.query.scan.ScanResultValue;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.hamcrest.Matchers;
import org.joda.time.Interval;

View File

@ -19,6 +19,7 @@
package org.apache.druid.indexing.common.task.batch.parallel.distribution;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;

View File

@ -25,6 +25,7 @@ import org.apache.datasketches.quantiles.ItemsSketch;
import org.apache.druid.jackson.JacksonModule;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.hamcrest.Matchers;
import org.hamcrest.number.IsCloseTo;
import org.junit.Assert;

View File

@ -54,13 +54,13 @@ import org.apache.druid.metadata.MetadataStorageTablesConfig;
import org.apache.druid.metadata.TestDerbyConnector;
import org.apache.druid.segment.TestHelper;
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.HashBasedNumberedShardSpecFactory;
import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory;
import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec;
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
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.ShardSpecFactory;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.Assert;
@ -949,8 +949,8 @@ public class TaskLockboxTest
{
final Task task = NoopTask.create();
lockbox.add(task);
allocateSegmentsAndAssert(task, "seq", 3, NumberedShardSpecFactory.instance());
allocateSegmentsAndAssert(task, "seq2", 2, new NumberedOverwritingShardSpecFactory(0, 3, (short) 1));
allocateSegmentsAndAssert(task, "seq", 3, NumberedPartialShardSpec.instance());
allocateSegmentsAndAssert(task, "seq2", 2, new NumberedOverwritePartialShardSpec(0, 3, (short) 1));
final List<TaskLock> locks = lockbox.findLocksForTask(task);
Assert.assertEquals(5, locks.size());
@ -971,15 +971,15 @@ public class TaskLockboxTest
final Task task = NoopTask.create();
lockbox.add(task);
allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedShardSpecFactory(null, 3));
allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedShardSpecFactory(null, 5));
allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedPartialShardSpec(null, 3));
allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedPartialShardSpec(null, 5));
}
private void allocateSegmentsAndAssert(
Task task,
String baseSequenceName,
int numSegmentsToAllocate,
ShardSpecFactory shardSpecFactory
PartialShardSpec partialShardSpec
)
{
for (int i = 0; i < numSegmentsToAllocate; i++) {
@ -988,7 +988,7 @@ public class TaskLockboxTest
TaskLockType.EXCLUSIVE,
task,
Intervals.of("2015-01-01/2015-01-05"),
shardSpecFactory,
partialShardSpec,
StringUtils.format("%s_%d", baseSequenceName, i),
null,
true
@ -1013,7 +1013,7 @@ public class TaskLockboxTest
Assert.assertEquals(lockRequest.getGroupId(), segmentLock.getGroupId());
Assert.assertEquals(lockRequest.getDataSource(), segmentLock.getDataSource());
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());
}

View File

@ -31,7 +31,7 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
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 java.util.ArrayList;
@ -144,7 +144,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
String sequenceName,
String previousSegmentId,
Interval interval,
ShardSpecFactory shardSpecFactory,
PartialShardSpec partialShardSpec,
String maxVersion,
boolean skipSegmentLineageCheck
)
@ -153,7 +153,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
dataSource,
interval,
maxVersion,
shardSpecFactory.create(objectMapper, 0)
partialShardSpec.complete(objectMapper, 0)
);
}

View File

@ -22,7 +22,7 @@ package org.apache.druid.indexing.overlord;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
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 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
* 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
* @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.
@ -171,7 +171,7 @@ public interface IndexerMetadataStorageCoordinator
String sequenceName,
@Nullable String previousSegmentId,
Interval interval,
ShardSpecFactory shardSpecFactory,
PartialShardSpec partialShardSpec,
String maxVersion,
boolean skipSegmentLineageCheck
);

View File

@ -48,9 +48,9 @@ import org.apache.druid.timeline.Partitions;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline;
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.ShardSpec;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.joda.time.Interval;
import org.skife.jdbi.v2.Folder3;
import org.skife.jdbi.v2.Handle;
@ -494,7 +494,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final String sequenceName,
@Nullable final String previousSegmentId,
final Interval interval,
final ShardSpecFactory shardSpecFactory,
final PartialShardSpec partialShardSpec,
final String maxVersion,
final boolean skipSegmentLineageCheck
)
@ -512,7 +512,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
dataSource,
sequenceName,
interval,
shardSpecFactory,
partialShardSpec,
maxVersion
);
} else {
@ -522,7 +522,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
sequenceName,
previousSegmentId,
interval,
shardSpecFactory,
partialShardSpec,
maxVersion
);
}
@ -537,7 +537,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final String sequenceName,
@Nullable final String previousSegmentId,
final Interval interval,
final ShardSpecFactory shardSpecFactory,
final PartialShardSpec partialShardSpec,
final String maxVersion
) throws IOException
{
@ -569,7 +569,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
handle,
dataSource,
interval,
shardSpecFactory,
partialShardSpec,
maxVersion
);
if (newIdentifier == null) {
@ -611,7 +611,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final String dataSource,
final String sequenceName,
final Interval interval,
final ShardSpecFactory shardSpecFactory,
final PartialShardSpec partialShardSpec,
final String maxVersion
) throws IOException
{
@ -645,7 +645,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
handle,
dataSource,
interval,
shardSpecFactory,
partialShardSpec,
maxVersion
);
if (newIdentifier == null) {
@ -788,7 +788,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final Handle handle,
final String dataSource,
final Interval interval,
final ShardSpecFactory shardSpecFactory,
final PartialShardSpec partialShardSpec,
final String maxVersion
) throws IOException
{
@ -812,8 +812,8 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
if (existingChunks
.stream()
.flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false))
.anyMatch(chunk -> !chunk.getObject().getShardSpec().isCompatible(shardSpecFactory.getShardSpecClass()))) {
// All existing segments should have a compatible shardSpec with shardSpecFactory.
.anyMatch(chunk -> !chunk.getObject().getShardSpec().isCompatible(partialShardSpec.getShardSpecClass()))) {
// All existing segments should have a compatible shardSpec with partialShardSpec.
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.
// Note that OverwriteShardSpec has the higher range for partitionId than others.
// 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()))
.map(chunk -> SegmentIdWithShardSpec.fromDataSegment(chunk.getObject()))
.orElse(null);
@ -845,7 +845,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
}
maxId = pendings.stream()
.filter(id -> id.getShardSpec().getClass() == shardSpecFactory.getShardSpecClass())
.filter(id -> id.getShardSpec().getClass() == partialShardSpec.getShardSpecClass())
.max((id1, id2) -> {
final int versionCompare = id1.getVersion().compareTo(id2.getVersion());
if (versionCompare != 0) {
@ -867,7 +867,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
}
if (maxId == null) {
final ShardSpec shardSpec = shardSpecFactory.create(jsonMapper, null);
final ShardSpec shardSpec = partialShardSpec.complete(jsonMapper, null);
String version = versionOfExistingChunks == null ? maxVersion : versionOfExistingChunks;
return new SegmentIdWithShardSpec(dataSource, interval, version, shardSpec);
} else if (!maxId.getInterval().equals(interval) || maxId.getVersion().compareTo(maxVersion) > 0) {
@ -880,7 +880,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
);
return null;
} else {
final ShardSpec newShardSpec = shardSpecFactory.create(jsonMapper, maxId.getShardSpec());
final ShardSpec newShardSpec = partialShardSpec.complete(jsonMapper, maxId.getShardSpec());
return new SegmentIdWithShardSpec(
dataSource,
maxId.getInterval(),

View File

@ -21,25 +21,32 @@ package org.apache.druid.segment.realtime.appenderator;
import org.apache.druid.data.input.InputRow;
import javax.annotation.Nullable;
import java.io.IOException;
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 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.
* 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
*/
@Nullable
SegmentIdWithShardSpec allocate(
InputRow row,
String sequenceName,
String previousSegmentId,
@Nullable String previousSegmentId,
boolean skipSegmentLineageCheck
) throws IOException;
}

View File

@ -34,16 +34,16 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
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.HashBasedNumberedShardSpecFactory;
import org.apache.druid.timeline.partition.LinearShardSpec;
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.NumberedOverwritingShardSpecFactory;
import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
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.ShardSpecFactory;
import org.assertj.core.api.Assertions;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -853,7 +853,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
@Test
public void testAllocatePendingSegment()
{
final ShardSpecFactory shardSpecFactory = NumberedShardSpecFactory.instance();
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment(
@ -861,7 +861,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
null,
interval,
shardSpecFactory,
partialShardSpec,
"version",
false
);
@ -873,7 +873,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
identifier.toString(),
interval,
shardSpecFactory,
partialShardSpec,
identifier.getVersion(),
false
);
@ -885,7 +885,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
identifier1.toString(),
interval,
shardSpecFactory,
partialShardSpec,
identifier1.getVersion(),
false
);
@ -897,7 +897,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
identifier1.toString(),
interval,
shardSpecFactory,
partialShardSpec,
identifier1.getVersion(),
false
);
@ -910,7 +910,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq1",
null,
interval,
shardSpecFactory,
partialShardSpec,
"version",
false
);
@ -921,7 +921,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
@Test
public void testDeletePendingSegment() throws InterruptedException
{
final ShardSpecFactory shardSpecFactory = NumberedShardSpecFactory.instance();
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
String prevSegmentId = null;
@ -934,7 +934,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
prevSegmentId,
interval,
shardSpecFactory,
partialShardSpec,
"version",
false
);
@ -949,7 +949,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
prevSegmentId,
interval,
shardSpecFactory,
partialShardSpec,
"version",
false
);
@ -973,7 +973,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
prevSegmentId,
interval,
new NumberedOverwritingShardSpecFactory(0, 1, (short) (i + 1)),
new NumberedOverwritePartialShardSpec(0, 1, (short) (i + 1)),
"version",
false
);
@ -1032,10 +1032,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
@Test
public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IOException
{
final ShardSpecFactory shardSpecFactory = new HashBasedNumberedShardSpecFactory(
null,
5
);
final PartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(null, 5);
final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01");
@ -1044,7 +1041,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq",
null,
interval,
shardSpecFactory,
partialShardSpec,
"version",
true
);
@ -1074,7 +1071,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq2",
null,
interval,
shardSpecFactory,
partialShardSpec,
"version",
true
);
@ -1104,7 +1101,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest
"seq3",
null,
interval,
new HashBasedNumberedShardSpecFactory(null, 3),
new HashBasedNumberedPartialShardSpec(null, 3),
"version",
true
);

View File

@ -36,6 +36,8 @@ import org.junit.Test;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class HashBasedNumberedShardSpecTest
{
@ -126,7 +128,6 @@ public class HashBasedNumberedShardSpecTest
@Test
public void testIsInChunk()
{
List<ShardSpec> specs = new ArrayList<>();
for (int i = 0; i < 3; i++) {
specs.add(new HashOverridenShardSpec(i, 3));
@ -137,7 +138,20 @@ public class HashBasedNumberedShardSpecTest
assertExistsInOneSpec(specs, new HashInputRow(0));
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
@ -186,10 +200,7 @@ public class HashBasedNumberedShardSpecTest
public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec
{
public HashOverridenShardSpec(
int partitionNum,
int partitions
)
public HashOverridenShardSpec(int partitionNum, int partitions)
{
super(partitionNum, partitions, null, ServerTestHelper.MAPPER);
}