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> <artifactId>jetty-util</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>nl.jqno.equalsverifier</groupId>
<artifactId>equalsverifier</artifactId>
<scope>test</scope>
</dependency>
<dependency> <dependency>
<groupId>com.google.errorprone</groupId> <groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId> <artifactId>error_prone_annotations</artifactId>

View File

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

View File

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

View File

@ -41,6 +41,9 @@ public interface PartitionsSpec
String MAX_ROWS_PER_SEGMENT = "maxRowsPerSegment"; String MAX_ROWS_PER_SEGMENT = "maxRowsPerSegment";
int HISTORICAL_NULL = -1; int HISTORICAL_NULL = -1;
@JsonIgnore
SecondaryPartitionType getType();
/** /**
* Returns the max number of rows per segment. * Returns the max number of rows per segment.
* Implementations can have different default values which it could be even null. * Implementations can have different default values which it could be even null.
@ -55,28 +58,6 @@ public interface PartitionsSpec
*/ */
boolean needsDeterminePartitions(boolean useForHadoopTask); boolean needsDeterminePartitions(boolean useForHadoopTask);
/**
* '-1' regarded as null for some historical reason.
*/
static boolean isEffectivelyNull(@Nullable Integer val)
{
return val == null || val == HISTORICAL_NULL;
}
/**
* '-1' regarded as null for some historical reason.
*/
static boolean isEffectivelyNull(@Nullable Long val)
{
return val == null || val == HISTORICAL_NULL;
}
@Nullable
static Integer resolveHistoricalNullIfNeeded(@Nullable Integer val)
{
return isEffectivelyNull(val) ? null : val;
}
/** /**
* @return True if this partitionSpec's type is compatible with forceGuaranteedRollup=true. * @return True if this partitionSpec's type is compatible with forceGuaranteedRollup=true.
*/ */
@ -101,4 +82,26 @@ public interface PartitionsSpec
*/ */
@JsonIgnore @JsonIgnore
String getForceGuaranteedRollupIncompatiblityReason(); String getForceGuaranteedRollupIncompatiblityReason();
/**
* '-1' regarded as null for some historical reason.
*/
static boolean isEffectivelyNull(@Nullable Integer val)
{
return val == null || val == HISTORICAL_NULL;
}
/**
* '-1' regarded as null for some historical reason.
*/
static boolean isEffectivelyNull(@Nullable Long val)
{
return val == null || val == HISTORICAL_NULL;
}
@Nullable
static Integer resolveHistoricalNullIfNeeded(@Nullable Integer val)
{
return isEffectivelyNull(val) ? null : val;
}
} }

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; return targetRowsPerSegment;
} }
@Override
public SecondaryPartitionType getType()
{
return SecondaryPartitionType.RANGE;
}
@JsonIgnore @JsonIgnore
@Override @Override
@NotNull @NotNull

View File

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

View File

@ -23,21 +23,21 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import javax.annotation.Nullable; import javax.annotation.Nullable;
public class LinearShardSpecFactory implements ShardSpecFactory public class LinearPartialShardSpec implements PartialShardSpec
{ {
private static final LinearShardSpecFactory INSTANCE = new LinearShardSpecFactory(); private static final LinearPartialShardSpec INSTANCE = new LinearPartialShardSpec();
public static LinearShardSpecFactory instance() public static LinearPartialShardSpec instance()
{ {
return INSTANCE; return INSTANCE;
} }
private LinearShardSpecFactory() private LinearPartialShardSpec()
{ {
} }
@Override @Override
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
{ {
return new LinearShardSpec( return new LinearShardSpec(
specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1 specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1
@ -45,7 +45,7 @@ public class LinearShardSpecFactory implements ShardSpecFactory
} }
@Override @Override
public ShardSpec create(ObjectMapper objectMapper, int partitionId) public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
{ {
return new LinearShardSpec(partitionId); return new LinearShardSpec(partitionId);
} }

View File

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

View File

@ -23,21 +23,21 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import javax.annotation.Nullable; import javax.annotation.Nullable;
public class NumberedShardSpecFactory implements ShardSpecFactory public class NumberedPartialShardSpec implements PartialShardSpec
{ {
private static final NumberedShardSpecFactory INSTANCE = new NumberedShardSpecFactory(); private static final NumberedPartialShardSpec INSTANCE = new NumberedPartialShardSpec();
public static NumberedShardSpecFactory instance() public static NumberedPartialShardSpec instance()
{ {
return INSTANCE; return INSTANCE;
} }
private NumberedShardSpecFactory() private NumberedPartialShardSpec()
{ {
} }
@Override @Override
public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
{ {
if (specOfPreviousMaxPartitionId == null) { if (specOfPreviousMaxPartitionId == null) {
return new NumberedShardSpec(0, 0); return new NumberedShardSpec(0, 0);
@ -48,7 +48,7 @@ public class NumberedShardSpecFactory implements ShardSpecFactory
} }
@Override @Override
public ShardSpec create(ObjectMapper objectMapper, int partitionId) public ShardSpec complete(ObjectMapper objectMapper, int partitionId)
{ {
return new NumberedShardSpec(partitionId, 0); return new NumberedShardSpec(partitionId, 0);
} }

View File

@ -19,38 +19,43 @@
package org.apache.druid.timeline.partition; package org.apache.druid.timeline.partition;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import javax.annotation.Nullable; import javax.annotation.Nullable;
/** /**
* Factory to be used to allocate segments remotely in the overlord. * Class to contain all information of a {@link ShardSpec} except for the partition ID.
* This class is mainly used by the indexing tasks to allocate new segments using the Overlord.
*/ */
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes({ @JsonSubTypes({
@JsonSubTypes.Type(name = "numbered", value = NumberedShardSpecFactory.class), @Type(name = "numbered", value = NumberedPartialShardSpec.class),
@JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpecFactory.class), @Type(name = HashBasedNumberedPartialShardSpec.TYPE, value = HashBasedNumberedPartialShardSpec.class),
@JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwritingShardSpecFactory.class), @Type(name = "single_dim", value = SingleDimensionPartialShardSpec.class),
@Type(name = "numbered_overwrite", value = NumberedOverwritePartialShardSpec.class),
}) })
public interface ShardSpecFactory public interface PartialShardSpec
{ {
/** /**
* Create a new shardSpec based on {@code specOfPreviousMaxPartitionId}. If it's null, it assumes that this is the * Creates a new ShardSpec based on {@code specOfPreviousMaxPartitionId}. If it's null, it assumes that this is the
* first call for the timeChunk where the new segment is created. * first call for the time chunk where the new segment is created.
* Note that {@code specOfPreviousMaxPartitionId} can also be null for {@link OverwriteShardSpec} if all segments * Note that {@code specOfPreviousMaxPartitionId} can also be null for {@link OverwriteShardSpec} if all segments
* in the timeChunk are first-generation segments. * in the timeChunk are first-generation segments.
*/ */
ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId); ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId);
/** /**
* Create a new shardSpec having the given partitionId. * Creates a new shardSpec having the given partitionId.
*/ */
ShardSpec create(ObjectMapper objectMapper, int partitionId); ShardSpec complete(ObjectMapper objectMapper, int partitionId);
/** /**
* Return the class of the shardSpec created by this factory. * Returns the class of the shardSpec created by this factory.
*/ */
@JsonIgnore
Class<? extends ShardSpec> getShardSpecClass(); Class<? extends ShardSpec> getShardSpecClass();
} }

View File

@ -17,7 +17,7 @@
* under the License. * under the License.
*/ */
package org.apache.druid.indexing.common.task.batch.parallel.distribution; package org.apache.druid.timeline.partition;
import com.google.common.collect.ForwardingList; import com.google.common.collect.ForwardingList;
@ -25,6 +25,7 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
@ -74,4 +75,31 @@ public class PartitionBoundaries extends ForwardingList<String> implements List<
{ {
return delegate; return delegate;
} }
public int getNumBuckets()
{
return delegate.size() - 1;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
PartitionBoundaries strings = (PartitionBoundaries) o;
return Objects.equals(delegate, strings.delegate);
}
@Override
public int hashCode()
{
return Objects.hash(super.hashCode(), delegate);
}
} }

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. * under the License.
*/ */
package org.apache.druid.indexing.common.task.batch.parallel.distribution; package org.apache.druid.timeline.partition;
import org.apache.druid.segment.TestHelper; import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -74,8 +76,23 @@ public class PartitionBoundariesTest
} }
@Test @Test
public void serializesDeserializes() public void serializesDeserializes() throws JsonProcessingException
{ {
TestHelper.testSerializesDeserializes(TestHelper.JSON_MAPPER, target); final ObjectMapper objectMapper = new ObjectMapper();
String serialized = objectMapper.writeValueAsString(target);
Object deserialized = objectMapper.readValue(serialized, target.getClass());
Assert.assertEquals(serialized, objectMapper.writeValueAsString(deserialized));
}
@Test
public void testGetNumBucketsOfNonEmptyPartitionBoundariesReturningCorrectSize()
{
Assert.assertEquals(2, target.getNumBuckets());
}
@Test
public void testEqualsContract()
{
EqualsVerifier.forClass(PartitionBoundaries.class).withNonnullFields("delegate").usingGetClass().verify();
} }
} }

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 StringInputRowParser(
new JSONParseSpec( new JSONParseSpec(
new TimestampSpec("ts", "yyyyMMddHH", null), new TimestampSpec("ts", "yyyyMMddHH", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("B", "F", "M", "Q", "X", "Y")), null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of(
"B",
"F",
"M",
"Q",
"X",
"Y"
)), null, null),
null, null,
null null
), ),
@ -553,12 +560,22 @@ public class IndexGeneratorJobTest
List<ShardSpec> specs = new ArrayList<>(); List<ShardSpec> specs = new ArrayList<>();
if ("hashed".equals(partitionType)) { if ("hashed".equals(partitionType)) {
for (Integer[] shardInfo : (Integer[][]) shardInfoForEachShard) { for (Integer[] shardInfo : (Integer[][]) shardInfoForEachShard) {
specs.add(new HashBasedNumberedShardSpec(shardInfo[0], shardInfo[1], null, HadoopDruidIndexerConfig.JSON_MAPPER)); specs.add(new HashBasedNumberedShardSpec(
shardInfo[0],
shardInfo[1],
null,
HadoopDruidIndexerConfig.JSON_MAPPER
));
} }
} else if ("single".equals(partitionType)) { } else if ("single".equals(partitionType)) {
int partitionNum = 0; int partitionNum = 0;
for (String[] shardInfo : (String[][]) shardInfoForEachShard) { for (String[] shardInfo : (String[][]) shardInfoForEachShard) {
specs.add(new SingleDimensionShardSpec("host", shardInfo[0], shardInfo[1], partitionNum++)); specs.add(new SingleDimensionShardSpec(
"host",
shardInfo[0],
shardInfo[1],
partitionNum++
));
} }
} else { } else {
throw new RE("Invalid partition type:[%s]", partitionType); throw new RE("Invalid partition type:[%s]", partitionType);

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

View File

@ -19,7 +19,6 @@
package org.apache.druid.indexing.common.actions; package org.apache.druid.indexing.common.actions;
import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.TaskStorage;
@ -27,15 +26,10 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
public class LocalTaskActionClient implements TaskActionClient public class LocalTaskActionClient implements TaskActionClient
{ {
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
private final ConcurrentHashMap<Class<? extends TaskAction>, AtomicInteger> actionCountMap = new ConcurrentHashMap<>();
private final Task task; private final Task task;
private final TaskStorage storage; private final TaskStorage storage;
private final TaskActionToolbox toolbox; private final TaskActionToolbox toolbox;
@ -79,17 +73,9 @@ public class LocalTaskActionClient implements TaskActionClient
final long performStartTime = System.currentTimeMillis(); final long performStartTime = System.currentTimeMillis();
final RetType result = taskAction.perform(task, toolbox); final RetType result = taskAction.perform(task, toolbox);
emitTimerMetric("task/action/run/time", System.currentTimeMillis() - performStartTime); emitTimerMetric("task/action/run/time", System.currentTimeMillis() - performStartTime);
actionCountMap.computeIfAbsent(taskAction.getClass(), k -> new AtomicInteger()).incrementAndGet();
return result; return result;
} }
@VisibleForTesting
public int getActionCount(Class<? extends TaskAction> actionClass)
{
final AtomicInteger count = actionCountMap.get(actionClass);
return count == null ? 0 : count.get();
}
private void emitTimerMetric(final String metric, final long time) private void emitTimerMetric(final String metric, final long time)
{ {
final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder(); final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder();

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.java.util.common.logger.Logger;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
import org.apache.druid.timeline.partition.ShardSpecFactory; import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -63,6 +63,8 @@ import java.util.stream.Collectors;
*/ */
public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec> public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
{ {
public static final String TYPE = "segmentAllocate";
private static final Logger log = new Logger(SegmentAllocateAction.class); private static final Logger log = new Logger(SegmentAllocateAction.class);
// Prevent spinning forever in situations where the segment list just won't stop changing. // Prevent spinning forever in situations where the segment list just won't stop changing.
@ -75,7 +77,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
private final String sequenceName; private final String sequenceName;
private final String previousSegmentId; private final String previousSegmentId;
private final boolean skipSegmentLineageCheck; private final boolean skipSegmentLineageCheck;
private final ShardSpecFactory shardSpecFactory; private final PartialShardSpec partialShardSpec;
private final LockGranularity lockGranularity; private final LockGranularity lockGranularity;
@JsonCreator @JsonCreator
@ -88,7 +90,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
@JsonProperty("previousSegmentId") String previousSegmentId, @JsonProperty("previousSegmentId") String previousSegmentId,
@JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck, @JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck,
// nullable for backward compatibility // nullable for backward compatibility
@JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory, @JsonProperty("shardSpecFactory") @Nullable PartialShardSpec partialShardSpec,
@JsonProperty("lockGranularity") @Nullable LockGranularity lockGranularity // nullable for backward compatibility @JsonProperty("lockGranularity") @Nullable LockGranularity lockGranularity // nullable for backward compatibility
) )
{ {
@ -102,7 +104,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName"); this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName");
this.previousSegmentId = previousSegmentId; this.previousSegmentId = previousSegmentId;
this.skipSegmentLineageCheck = skipSegmentLineageCheck; this.skipSegmentLineageCheck = skipSegmentLineageCheck;
this.shardSpecFactory = shardSpecFactory == null ? NumberedShardSpecFactory.instance() : shardSpecFactory; this.partialShardSpec = partialShardSpec == null ? NumberedPartialShardSpec.instance() : partialShardSpec;
this.lockGranularity = lockGranularity == null ? LockGranularity.TIME_CHUNK : lockGranularity; this.lockGranularity = lockGranularity == null ? LockGranularity.TIME_CHUNK : lockGranularity;
} }
@ -148,10 +150,10 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
return skipSegmentLineageCheck; return skipSegmentLineageCheck;
} }
@JsonProperty @JsonProperty("shardSpecFactory")
public ShardSpecFactory getShardSpecFactory() public PartialShardSpec getPartialShardSpec()
{ {
return shardSpecFactory; return partialShardSpec;
} }
@JsonProperty @JsonProperty
@ -293,7 +295,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
task.getGroupId(), task.getGroupId(),
dataSource, dataSource,
tryInterval, tryInterval,
shardSpecFactory, partialShardSpec,
task.getPriority(), task.getPriority(),
sequenceName, sequenceName,
previousSegmentId, previousSegmentId,
@ -355,7 +357,7 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
", sequenceName='" + sequenceName + '\'' + ", sequenceName='" + sequenceName + '\'' +
", previousSegmentId='" + previousSegmentId + '\'' + ", previousSegmentId='" + previousSegmentId + '\'' +
", skipSegmentLineageCheck=" + skipSegmentLineageCheck + ", skipSegmentLineageCheck=" + skipSegmentLineageCheck +
", shardSpecFactory=" + shardSpecFactory + ", partialShardSpec=" + partialShardSpec +
", lockGranularity=" + lockGranularity + ", lockGranularity=" + lockGranularity +
'}'; '}';
} }

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.LockGranularity;
import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.SegmentLockHelper;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.TaskLockHelper;
import org.apache.druid.indexing.overlord.CriticalAction; import org.apache.druid.indexing.overlord.CriticalAction;
import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.SegmentPublishResult;
@ -238,8 +238,8 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
final Map<Interval, List<DataSegment>> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten); final Map<Interval, List<DataSegment>> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten);
final Map<Interval, List<DataSegment>> newSegmentsMap = groupSegmentsByIntervalAndSort(segments); final Map<Interval, List<DataSegment>> newSegmentsMap = groupSegmentsByIntervalAndSort(segments);
oldSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); oldSegmentsMap.values().forEach(TaskLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
newSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); newSegmentsMap.values().forEach(TaskLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> { oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> {
final List<DataSegment> newSegmentsPerInterval = Preconditions.checkNotNull( final List<DataSegment> newSegmentsPerInterval = Preconditions.checkNotNull(

View File

@ -40,7 +40,7 @@ import org.apache.druid.indexing.common.task.Task;
@JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class), @JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class),
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class),
@JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class), @JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class),
@JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class), @JsonSubTypes.Type(name = SegmentAllocateAction.TYPE, value = SegmentAllocateAction.class),
@JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class), @JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class),
@JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class), @JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class),
@JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class) @JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class)

View File

@ -24,8 +24,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskLockType;
@ -40,7 +38,6 @@ import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.indexing.firehose.WindowedSegmentId;
import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.granularity.GranularityType;
import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.logger.Logger;
@ -48,24 +45,20 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.Partitions;
import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import javax.annotation.Nonnull;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.GuardedBy;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.SortedSet;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -78,24 +71,17 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
{ {
private static final Logger log = new Logger(AbstractBatchIndexTask.class); private static final Logger log = new Logger(AbstractBatchIndexTask.class);
private final SegmentLockHelper segmentLockHelper;
@GuardedBy("this") @GuardedBy("this")
private final TaskResourceCleaner resourceCloserOnAbnormalExit = new TaskResourceCleaner(); private final TaskResourceCleaner resourceCloserOnAbnormalExit = new TaskResourceCleaner();
/**
* State to indicate that this task will use segmentLock or timeChunkLock.
* This is automatically set when {@link #determineLockGranularityandTryLock} is called.
*/
private boolean useSegmentLock;
@GuardedBy("this") @GuardedBy("this")
private boolean stopped = false; private boolean stopped = false;
private TaskLockHelper taskLockHelper;
protected AbstractBatchIndexTask(String id, String dataSource, Map<String, Object> context) protected AbstractBatchIndexTask(String id, String dataSource, Map<String, Object> context)
{ {
super(id, dataSource, context); super(id, dataSource, context);
segmentLockHelper = new SegmentLockHelper();
} }
protected AbstractBatchIndexTask( protected AbstractBatchIndexTask(
@ -107,7 +93,6 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
) )
{ {
super(id, groupId, taskResource, dataSource, context); super(id, groupId, taskResource, dataSource, context);
segmentLockHelper = new SegmentLockHelper();
} }
/** /**
@ -194,14 +179,9 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
} }
public boolean isUseSegmentLock() public TaskLockHelper getTaskLockHelper()
{ {
return useSegmentLock; return Preconditions.checkNotNull(taskLockHelper, "taskLockHelper is not initialized yet");
}
public SegmentLockHelper getSegmentLockHelper()
{
return segmentLockHelper;
} }
/** /**
@ -229,7 +209,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
// Respect task context value most. // Respect task context value most.
if (forceTimeChunkLock) { if (forceTimeChunkLock) {
log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY); log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
useSegmentLock = false; taskLockHelper = new TaskLockHelper(false);
if (!intervals.isEmpty()) { if (!intervals.isEmpty()) {
return tryTimeChunkLock(client, intervals); return tryTimeChunkLock(client, intervals);
} else { } else {
@ -238,7 +218,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
} else { } else {
if (!intervals.isEmpty()) { if (!intervals.isEmpty()) {
final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals); final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals);
useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT; taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT);
return tryLockWithDetermineResult(client, result); return tryLockWithDetermineResult(client, result);
} else { } else {
return true; return true;
@ -255,14 +235,14 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
); );
if (forceTimeChunkLock) { if (forceTimeChunkLock) {
log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY); log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
useSegmentLock = false; taskLockHelper = new TaskLockHelper(false);
return tryTimeChunkLock( return tryTimeChunkLock(
client, client,
new ArrayList<>(segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet())) new ArrayList<>(segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()))
); );
} else { } else {
final LockGranularityDetermineResult result = determineSegmentGranularity(segments); final LockGranularityDetermineResult result = determineSegmentGranularity(segments);
useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT; taskLockHelper = new TaskLockHelper(result.lockGranularity == LockGranularity.SEGMENT);
return tryLockWithDetermineResult(client, result); return tryLockWithDetermineResult(client, result);
} }
} }
@ -298,7 +278,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
if (result.lockGranularity == LockGranularity.TIME_CHUNK) { if (result.lockGranularity == LockGranularity.TIME_CHUNK) {
return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals")); return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals"));
} else { } else {
return segmentLockHelper.verifyAndLockExistingSegments( return taskLockHelper.verifyAndLockExistingSegments(
client, client,
Preconditions.checkNotNull(result.segments, "segments") Preconditions.checkNotNull(result.segments, "segments")
); );
@ -377,7 +357,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
/** /**
* We currently don't support appending perfectly rolled up segments. This might be supported in the future if there * We currently don't support appending perfectly rolled up segments. This might be supported in the future if there
* is a good use case. If we want to support appending perfectly rolled up segments, we need to fix some other places * is a good use case. If we want to support appending perfectly rolled up segments, we need to fix some other places
* first. For example, {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec#getLookup} assumes that * first. For example, {@link HashBasedNumberedShardSpec#getLookup} assumes that
* the start partition ID of the set of perfectly rolled up segments is 0. Instead it might need to store an ordinal * the start partition ID of the set of perfectly rolled up segments is 0. Instead it might need to store an ordinal
* in addition to the partition ID which represents the ordinal in the perfectly rolled up segment set. * in addition to the partition ID which represents the ordinal in the perfectly rolled up segment set.
*/ */
@ -390,14 +370,6 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
return tuningConfig.isForceGuaranteedRollup(); return tuningConfig.isForceGuaranteedRollup();
} }
static Pair<ShardSpecFactory, Integer> createShardSpecFactoryForGuaranteedRollup(
int numShards,
@Nullable List<String> partitionDimensions
)
{
return Pair.of(new HashBasedNumberedShardSpecFactory(partitionDimensions, numShards), numShards);
}
@Nullable @Nullable
static Granularity findGranularityFromSegments(List<DataSegment> segments) static Granularity findGranularityFromSegments(List<DataSegment> segments)
{ {
@ -415,45 +387,6 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
} }
} }
/**
* Creates shard specs based on the given configurations. The return value is a map between intervals created
* based on the segment granularity and the shard specs to be created.
* Note that the shard specs to be created is a pair of {@link ShardSpecFactory} and number of segments per interval
* and filled only when {@link #isGuaranteedRollup} = true. Otherwise, the return value contains only the set of
* intervals generated based on the segment granularity.
*/
protected static Map<Interval, Pair<ShardSpecFactory, Integer>> createShardSpecWithoutInputScan(
GranularitySpec granularitySpec,
IndexIOConfig ioConfig,
IndexTuningConfig tuningConfig,
@Nonnull PartitionsSpec nonNullPartitionsSpec
)
{
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = new HashMap<>();
final SortedSet<Interval> intervals = granularitySpec.bucketIntervals().get();
if (isGuaranteedRollup(ioConfig, tuningConfig)) {
// SingleDimensionPartitionsSpec or more partitionsSpec types will be supported in the future.
assert nonNullPartitionsSpec instanceof HashedPartitionsSpec;
// Overwrite mode, guaranteed rollup: shardSpecs must be known in advance.
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) nonNullPartitionsSpec;
final int numShards = partitionsSpec.getNumShards() == null ? 1 : partitionsSpec.getNumShards();
for (Interval interval : intervals) {
allocateSpec.put(
interval,
createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions())
);
}
} else {
for (Interval interval : intervals) {
allocateSpec.put(interval, null);
}
}
return allocateSpec;
}
/** /**
* If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock * If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock
* from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be * from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be

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

View File

@ -19,17 +19,21 @@
package org.apache.druid.indexing.common.task; package org.apache.druid.indexing.common.task;
import com.google.common.base.Preconditions;
import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.LockListAction;
import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.SurrogateAction;
import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs; import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs;
import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
@ -41,11 +45,8 @@ import java.util.stream.Collectors;
/** /**
* Allocates all necessary segments locally at the beginning and reuses them. * Allocates all necessary segments locally at the beginning and reuses them.
*
* @see HashPartitionCachingLocalSegmentAllocator
* @see RangePartitionCachingLocalSegmentAllocator
*/ */
class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator public class CachingLocalSegmentAllocator implements CachingSegmentAllocator
{ {
private final String taskId; private final String taskId;
private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId; private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
@ -59,22 +60,34 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
* *
* @return Information for segment preallocation * @return Information for segment preallocation
*/ */
Map<Interval, List<SegmentIdWithShardSpec>> create(Function<Interval, String> versionFinder); Map<Interval, List<SegmentIdWithShardSpec>> create(
TaskToolbox toolbox,
String dataSource,
Function<Interval, String> versionFinder
);
} }
CachingLocalSegmentAllocatorHelper( CachingLocalSegmentAllocator(
TaskToolbox toolbox, TaskToolbox toolbox,
String dataSource,
String taskId, String taskId,
String supervisorTaskId, @Nullable SupervisorTaskAccess supervisorTaskAccess,
IntervalToSegmentIdsCreator intervalToSegmentIdsCreator IntervalToSegmentIdsCreator intervalToSegmentIdsCreator
) throws IOException ) throws IOException
{ {
this.taskId = taskId; this.taskId = taskId;
this.sequenceNameToSegmentId = new HashMap<>(); this.sequenceNameToSegmentId = new HashMap<>();
final TaskAction<List<TaskLock>> action;
if (supervisorTaskAccess == null) {
action = new LockListAction();
} else {
action = new SurrogateAction<>(supervisorTaskAccess.getSupervisorTaskId(), new LockListAction());
}
final Map<Interval, String> intervalToVersion = final Map<Interval, String> intervalToVersion =
toolbox.getTaskActionClient() toolbox.getTaskActionClient()
.submit(new SurrogateAction<>(supervisorTaskId, new LockListAction())) .submit(action)
.stream() .stream()
.collect(Collectors.toMap( .collect(Collectors.toMap(
TaskLock::getInterval, TaskLock::getInterval,
@ -82,7 +95,11 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
)); ));
Function<Interval, String> versionFinder = interval -> findVersion(intervalToVersion, interval); Function<Interval, String> versionFinder = interval -> findVersion(intervalToVersion, interval);
final Map<Interval, List<SegmentIdWithShardSpec>> intervalToIds = intervalToSegmentIdsCreator.create(versionFinder); final Map<Interval, List<SegmentIdWithShardSpec>> intervalToIds = intervalToSegmentIdsCreator.create(
toolbox,
dataSource,
versionFinder
);
final Map<Interval, List<ShardSpec>> shardSpecMap = new HashMap<>(); final Map<Interval, List<ShardSpec>> shardSpecMap = new HashMap<>();
for (Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) { for (Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) {
@ -115,14 +132,11 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
boolean skipSegmentLineageCheck boolean skipSegmentLineageCheck
) )
{ {
return sequenceNameToSegmentId.get(sequenceName); return Preconditions.checkNotNull(
} sequenceNameToSegmentId.get(sequenceName),
"Missing segmentId for the sequence[%s]",
@Override sequenceName
public String getSequenceName(Interval interval, InputRow inputRow) );
{
// Sequence name is based solely on the shardSpec, and there will only be one segment per sequence.
return getSequenceName(interval, shardSpecs.getShardSpec(interval, inputRow));
} }
/** /**
@ -136,4 +150,10 @@ class CachingLocalSegmentAllocatorHelper implements IndexTaskSegmentAllocator
// and it's faster to add strings together than it is to use String#format // and it's faster to add strings together than it is to use String#format
return taskId + "_" + interval + "_" + shardSpec.getPartitionNum(); return taskId + "_" + interval + "_" + shardSpec.getPartitionNum();
} }
@Override
public ShardSpecs getShardSpecs()
{
return shardSpecs;
}
} }

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.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SecondaryPartitionType;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskReport;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
@ -64,14 +64,19 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMeters;
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder;
import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis;
import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
import org.apache.druid.indexing.common.task.batch.partition.LinearPartitionAnalysis;
import org.apache.druid.indexing.common.task.batch.partition.PartitionAnalysis;
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.logger.Logger;
@ -94,6 +99,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandler;
@ -105,12 +111,12 @@ import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.apache.druid.utils.CircularBuffer; import org.apache.druid.utils.CircularBuffer;
import org.codehaus.plexus.util.FileUtils; import org.codehaus.plexus.util.FileUtils;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import javax.annotation.Nonnull;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.GET; import javax.ws.rs.GET;
@ -130,6 +136,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.SortedSet;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -264,6 +271,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@Override @Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception public boolean isReady(TaskActionClient taskActionClient) throws Exception
{ {
final IndexTuningConfig tuningConfig = getIngestionSchema().getTuningConfig();
if (tuningConfig != null && tuningConfig.getPartitionsSpec() != null) {
if (tuningConfig.getPartitionsSpec().getType() != SecondaryPartitionType.LINEAR
&& tuningConfig.getPartitionsSpec().getType() != SecondaryPartitionType.HASH) {
throw new UOE("partitionsSpec[%s] is not supported", tuningConfig.getPartitionsSpec().getClass().getName());
}
}
return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.dataSchema.getGranularitySpec()); return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.dataSchema.getGranularitySpec());
} }
@ -487,13 +501,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
// Initialize maxRowsPerSegment and maxTotalRows lazily // Initialize maxRowsPerSegment and maxTotalRows lazily
final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig; final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig;
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec(); final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = determineShardSpecs( final PartitionAnalysis partitionAnalysis = determineShardSpecs(
toolbox, toolbox,
inputSource, inputSource,
tmpDir, tmpDir,
partitionsSpec partitionsSpec
); );
final List<Interval> allocateIntervals = new ArrayList<>(allocateSpec.keySet()); final List<Interval> allocateIntervals = new ArrayList<>(partitionAnalysis.getAllIntervalsToIndex());
final DataSchema dataSchema; final DataSchema dataSchema;
if (determineIntervals) { if (determineIntervals) {
if (!determineLockGranularityandTryLock(toolbox.getTaskActionClient(), allocateIntervals)) { if (!determineLockGranularityandTryLock(toolbox.getTaskActionClient(), allocateIntervals)) {
@ -508,14 +522,14 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
} else { } else {
dataSchema = ingestionSchema.getDataSchema(); dataSchema = ingestionSchema.getDataSchema();
} }
ingestionState = IngestionState.BUILD_SEGMENTS; ingestionState = IngestionState.BUILD_SEGMENTS;
return generateAndPublishSegments( return generateAndPublishSegments(
toolbox, toolbox,
dataSchema, dataSchema,
allocateSpec,
inputSource, inputSource,
tmpDir, tmpDir,
partitionsSpec partitionAnalysis
); );
} }
catch (Exception e) { catch (Exception e) {
@ -598,16 +612,14 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
* *
* @return a map indicating how many shardSpecs need to be created per interval. * @return a map indicating how many shardSpecs need to be created per interval.
*/ */
private Map<Interval, Pair<ShardSpecFactory, Integer>> determineShardSpecs( private PartitionAnalysis determineShardSpecs(
final TaskToolbox toolbox, final TaskToolbox toolbox,
final InputSource inputSource, final InputSource inputSource,
final File tmpDir, final File tmpDir,
final PartitionsSpec nonNullPartitionsSpec @Nonnull final PartitionsSpec partitionsSpec
) throws IOException ) throws IOException
{ {
final ObjectMapper jsonMapper = toolbox.getJsonMapper(); final ObjectMapper jsonMapper = toolbox.getJsonMapper();
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final IndexIOConfig ioConfig = ingestionSchema.getIOConfig();
final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
@ -615,42 +627,59 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent(); final boolean determineIntervals = !granularitySpec.bucketIntervals().isPresent();
// Must determine partitions if rollup is guaranteed and the user didn't provide a specific value. // Must determine partitions if rollup is guaranteed and the user didn't provide a specific value.
final boolean determineNumPartitions = nonNullPartitionsSpec.needsDeterminePartitions(false); final boolean determineNumPartitions = partitionsSpec.needsDeterminePartitions(false);
// if we were given number of shards per interval and the intervals, we don't need to scan the data // if we were given number of shards per interval and the intervals, we don't need to scan the data
if (!determineNumPartitions && !determineIntervals) { if (!determineNumPartitions && !determineIntervals) {
log.info("Skipping determine partition scan"); log.info("Skipping determine partition scan");
return createShardSpecWithoutInputScan( if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
return PartialHashSegmentGenerateTask.createHashPartitionAnalysisFromPartitionsSpec(
granularitySpec, granularitySpec,
ioConfig, (HashedPartitionsSpec) partitionsSpec
tuningConfig,
nonNullPartitionsSpec
); );
} else if (partitionsSpec.getType() == SecondaryPartitionType.LINEAR) {
return createLinearPartitionAnalysis(granularitySpec, (DynamicPartitionsSpec) partitionsSpec);
} else {
throw new UOE("%s", partitionsSpec.getClass().getName());
}
} else { } else {
// determine intervals containing data and prime HLL collectors // determine intervals containing data and prime HLL collectors
log.info("Determining intervals and shardSpecs");
return createShardSpecsFromInput( return createShardSpecsFromInput(
jsonMapper, jsonMapper,
ingestionSchema, ingestionSchema,
inputSource, inputSource,
tmpDir, tmpDir,
granularitySpec, granularitySpec,
nonNullPartitionsSpec, partitionsSpec,
determineIntervals determineIntervals
); );
} }
} }
private Map<Interval, Pair<ShardSpecFactory, Integer>> createShardSpecsFromInput( private static LinearPartitionAnalysis createLinearPartitionAnalysis(
GranularitySpec granularitySpec,
@Nonnull DynamicPartitionsSpec partitionsSpec
)
{
final SortedSet<Interval> intervals = granularitySpec.bucketIntervals().get();
final int numBucketsPerInterval = 1;
final LinearPartitionAnalysis partitionAnalysis = new LinearPartitionAnalysis(partitionsSpec);
intervals.forEach(interval -> partitionAnalysis.updateBucket(interval, numBucketsPerInterval));
return partitionAnalysis;
}
private PartitionAnalysis createShardSpecsFromInput(
ObjectMapper jsonMapper, ObjectMapper jsonMapper,
IndexIngestionSpec ingestionSchema, IndexIngestionSpec ingestionSchema,
InputSource inputSource, InputSource inputSource,
File tmpDir, File tmpDir,
GranularitySpec granularitySpec, GranularitySpec granularitySpec,
PartitionsSpec nonNullPartitionsSpec, @Nonnull PartitionsSpec partitionsSpec,
boolean determineIntervals boolean determineIntervals
) throws IOException ) throws IOException
{ {
log.info("Determining intervals and shardSpecs"); assert partitionsSpec.getType() != SecondaryPartitionType.RANGE;
long determineShardSpecsStartMillis = System.currentTimeMillis(); long determineShardSpecsStartMillis = System.currentTimeMillis();
final Map<Interval, Optional<HyperLogLogCollector>> hllCollectors = collectIntervalsAndShardSpecs( final Map<Interval, Optional<HyperLogLogCollector>> hllCollectors = collectIntervalsAndShardSpecs(
@ -659,50 +688,49 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
inputSource, inputSource,
tmpDir, tmpDir,
granularitySpec, granularitySpec,
nonNullPartitionsSpec, partitionsSpec,
determineIntervals determineIntervals
); );
final PartitionAnalysis<Integer, ?> partitionAnalysis;
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpecs = new HashMap<>(); if (partitionsSpec.getType() == SecondaryPartitionType.LINEAR) {
partitionAnalysis = new LinearPartitionAnalysis((DynamicPartitionsSpec) partitionsSpec);
} else if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
partitionAnalysis = new HashPartitionAnalysis((HashedPartitionsSpec) partitionsSpec);
} else {
throw new UOE("%s", partitionsSpec.getClass().getName());
}
for (final Map.Entry<Interval, Optional<HyperLogLogCollector>> entry : hllCollectors.entrySet()) { for (final Map.Entry<Interval, Optional<HyperLogLogCollector>> entry : hllCollectors.entrySet()) {
final Interval interval = entry.getKey(); final Interval interval = entry.getKey();
final int numBucketsPerInterval;
if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) { if (partitionsSpec.getType() == SecondaryPartitionType.HASH) {
assert nonNullPartitionsSpec instanceof HashedPartitionsSpec; final HashedPartitionsSpec hashedPartitionsSpec = (HashedPartitionsSpec) partitionsSpec;
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) nonNullPartitionsSpec;
final HyperLogLogCollector collector = entry.getValue().orNull(); final HyperLogLogCollector collector = entry.getValue().orNull();
final int numShards;
if (partitionsSpec.needsDeterminePartitions(false)) { if (partitionsSpec.needsDeterminePartitions(false)) {
final long numRows = Preconditions.checkNotNull(collector, "HLL collector").estimateCardinalityRound(); final long numRows = Preconditions.checkNotNull(collector, "HLL collector").estimateCardinalityRound();
final int nonNullMaxRowsPerSegment = partitionsSpec.getMaxRowsPerSegment() == null final int nonNullMaxRowsPerSegment = partitionsSpec.getMaxRowsPerSegment() == null
? PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT ? PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT
: partitionsSpec.getMaxRowsPerSegment(); : partitionsSpec.getMaxRowsPerSegment();
numShards = (int) Math.ceil((double) numRows / nonNullMaxRowsPerSegment); numBucketsPerInterval = (int) Math.ceil((double) numRows / nonNullMaxRowsPerSegment);
log.info( log.info(
"Estimated [%,d] rows of data for interval [%s], creating [%,d] shards", "Estimated [%,d] rows of data for interval [%s], creating [%,d] shards",
numRows, numRows,
interval, interval,
numShards numBucketsPerInterval
); );
} else { } else {
numShards = partitionsSpec.getNumShards() == null ? 1 : partitionsSpec.getNumShards(); numBucketsPerInterval = hashedPartitionsSpec.getNumShards() == null ? 1 : hashedPartitionsSpec.getNumShards();
log.info("Creating [%,d] shards for interval [%s]", numShards, interval); log.info("Creating [%,d] buckets for interval [%s]", numBucketsPerInterval, interval);
}
} else {
numBucketsPerInterval = 1;
} }
// Overwrite mode, guaranteed rollup: shardSpecs must be known in advance. partitionAnalysis.updateBucket(interval, numBucketsPerInterval);
allocateSpecs.put(
interval,
createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions())
);
} else {
allocateSpecs.put(interval, null);
}
} }
log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis);
return allocateSpecs; return partitionAnalysis;
} }
private Map<Interval, Optional<HyperLogLogCollector>> collectIntervalsAndShardSpecs( private Map<Interval, Optional<HyperLogLogCollector>> collectIntervalsAndShardSpecs(
@ -711,7 +739,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
InputSource inputSource, InputSource inputSource,
File tmpDir, File tmpDir,
GranularitySpec granularitySpec, GranularitySpec granularitySpec,
PartitionsSpec nonNullPartitionsSpec, @Nonnull PartitionsSpec partitionsSpec,
boolean determineIntervals boolean determineIntervals
) throws IOException ) throws IOException
{ {
@ -765,7 +793,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
interval = optInterval.get(); interval = optInterval.get();
} }
if (nonNullPartitionsSpec.needsDeterminePartitions(false)) { if (partitionsSpec.needsDeterminePartitions(false)) {
hllCollectors.computeIfAbsent(interval, intv -> Optional.of(HyperLogLogCollector.makeLatestCollector())); hllCollectors.computeIfAbsent(interval, intv -> Optional.of(HyperLogLogCollector.makeLatestCollector()));
List<Object> groupKey = Rows.toGroupKey( List<Object> groupKey = Rows.toGroupKey(
@ -809,32 +837,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
return hllCollectors; return hllCollectors;
} }
private IndexTaskSegmentAllocator createSegmentAllocator(
TaskToolbox toolbox,
DataSchema dataSchema,
Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec
) throws IOException
{
if (ingestionSchema.ioConfig.isAppendToExisting() || isUseSegmentLock()) {
return new RemoteSegmentAllocator(
toolbox,
getId(),
dataSchema,
getSegmentLockHelper(),
isUseSegmentLock() ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK,
ingestionSchema.ioConfig.isAppendToExisting()
);
} else {
// We use the timeChunk lock and don't have to ask the overlord to create segmentIds.
// Instead, a local allocator is used.
if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) {
return new HashPartitionCachingLocalSegmentAllocator(toolbox, getId(), getId(), getDataSource(), allocateSpec);
} else {
return new LocalSegmentAllocator(toolbox, getId(), getDataSource(), dataSchema.getGranularitySpec());
}
}
}
/** /**
* This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}. * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
* If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs * If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs
@ -856,10 +858,9 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
private TaskStatus generateAndPublishSegments( private TaskStatus generateAndPublishSegments(
final TaskToolbox toolbox, final TaskToolbox toolbox,
final DataSchema dataSchema, final DataSchema dataSchema,
final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec,
final InputSource inputSource, final InputSource inputSource,
final File tmpDir, final File tmpDir,
final PartitionsSpec partitionsSpec final PartitionAnalysis partitionAnalysis
) throws IOException, InterruptedException ) throws IOException, InterruptedException
{ {
final FireDepartment fireDepartmentForMetrics = final FireDepartment fireDepartmentForMetrics =
@ -875,14 +876,42 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
toolbox.getMonitorScheduler().addMonitor(metricsMonitor); toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
} }
final PartitionsSpec partitionsSpec = partitionAnalysis.getPartitionsSpec();
final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final long pushTimeout = tuningConfig.getPushTimeout(); final long pushTimeout = tuningConfig.getPushTimeout();
final IndexTaskSegmentAllocator segmentAllocator = createSegmentAllocator( final SegmentAllocator segmentAllocator;
final SequenceNameFunction sequenceNameFunction;
switch (partitionsSpec.getType()) {
case HASH:
case RANGE:
final CachingSegmentAllocator localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning(
toolbox, toolbox,
dataSchema, getDataSource(),
allocateSpec getId(),
null,
(CompletePartitionAnalysis) partitionAnalysis
); );
sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(
getId(),
localSegmentAllocator.getShardSpecs()
);
segmentAllocator = localSegmentAllocator;
break;
case LINEAR:
segmentAllocator = SegmentAllocators.forLinearPartitioning(
toolbox,
null,
dataSchema,
getTaskLockHelper(),
ingestionSchema.getIOConfig().isAppendToExisting(),
partitionAnalysis.getPartitionsSpec()
);
sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(getId());
break;
default:
throw new UOE("[%s] secondary partition type is not supported", partitionsSpec.getType());
}
final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) ->
toolbox.getTaskActionClient() toolbox.getTaskActionClient()
@ -921,13 +950,13 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
inputSource, inputSource,
inputSource.needsFormat() ? getInputFormat(ingestionSchema) : null, inputSource.needsFormat() ? getInputFormat(ingestionSchema) : null,
tmpDir, tmpDir,
segmentAllocator sequenceNameFunction
); );
// If we use timeChunk lock, then we don't have to specify what segments will be overwritten because // If we use timeChunk lock, then we don't have to specify what segments will be overwritten because
// it will just overwrite all segments overlapped with the new segments. // it will just overwrite all segments overlapped with the new segments.
final Set<DataSegment> inputSegments = isUseSegmentLock() final Set<DataSegment> inputSegments = getTaskLockHelper().isUseSegmentLock()
? getSegmentLockHelper().getLockedExistingSegments() ? getTaskLockHelper().getLockedExistingSegments()
: null; : null;
// Probably we can publish atomicUpdateGroup along with segments. // Probably we can publish atomicUpdateGroup along with segments.
final SegmentsAndCommitMetadata published = final SegmentsAndCommitMetadata published =
@ -1207,11 +1236,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
@Nullable @Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
static IndexTuningConfig createDefault()
{
return new IndexTuningConfig();
}
@Nullable @Nullable
private static PartitionsSpec getDefaultPartitionsSpec( private static PartitionsSpec getDefaultPartitionsSpec(
boolean forceGuaranteedRollup, boolean forceGuaranteedRollup,

View File

@ -96,7 +96,7 @@ public class InputSourceProcessor
InputSource inputSource, InputSource inputSource,
@Nullable InputFormat inputFormat, @Nullable InputFormat inputFormat,
File tmpDir, File tmpDir,
IndexTaskSegmentAllocator segmentAllocator SequenceNameFunction sequenceNameFunction
) throws IOException, InterruptedException, ExecutionException, TimeoutException ) throws IOException, InterruptedException, ExecutionException, TimeoutException
{ {
@Nullable @Nullable
@ -140,7 +140,7 @@ public class InputSourceProcessor
@SuppressWarnings("OptionalGetWithoutIsPresent") @SuppressWarnings("OptionalGetWithoutIsPresent")
final Interval interval = optInterval.get(); final Interval interval = optInterval.get();
final String sequenceName = segmentAllocator.getSequenceName(interval, inputRow); final String sequenceName = sequenceNameFunction.getSequenceName(interval, inputRow);
final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName);
if (addResult.isOk()) { if (addResult.isOk()) {

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; package org.apache.druid.indexing.common.task;
import com.google.common.base.Optional; import com.google.common.base.Optional;
import com.google.common.collect.Maps;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
@ -32,32 +34,27 @@ import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
* Segment allocator which allocates new segments locally per request. * Segment allocator which allocates new segments locally per request.
*/ */
class LocalSegmentAllocator implements IndexTaskSegmentAllocator class LocalSegmentAllocator implements SegmentAllocator
{ {
private final String taskId;
private final SegmentAllocator internalAllocator; private final SegmentAllocator internalAllocator;
LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec) LocalSegmentAllocator(TaskToolbox toolbox, String dataSource, GranularitySpec granularitySpec) throws IOException
throws IOException
{ {
this.taskId = taskId; final Map<Interval, String> intervalToVersion = toolbox
final Map<Interval, AtomicInteger> counters = new HashMap<>(); .getTaskActionClient()
final Map<Interval, String> intervalToVersion = toolbox.getTaskActionClient()
.submit(new LockListAction()) .submit(new LockListAction())
.stream() .stream()
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
final Map<Interval, MutableInt> counters = Maps.newHashMapWithExpectedSize(intervalToVersion.size());
internalAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> { internalAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
final DateTime timestamp = row.getTimestamp(); final DateTime timestamp = row.getTimestamp();
@ -67,30 +64,25 @@ class LocalSegmentAllocator implements IndexTaskSegmentAllocator
} }
final Interval interval = maybeInterval.get(); final Interval interval = maybeInterval.get();
final String version = intervalToVersion.entrySet().stream() final String version = intervalToVersion
.entrySet()
.stream()
.filter(entry -> entry.getKey().contains(interval)) .filter(entry -> entry.getKey().contains(interval))
.map(Entry::getValue) .map(Entry::getValue)
.findFirst() .findFirst()
.orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
final int partitionNum = counters.computeIfAbsent(interval, x -> new AtomicInteger()).getAndIncrement(); final int partitionId = counters.computeIfAbsent(interval, x -> new MutableInt()).getAndIncrement();
return new SegmentIdWithShardSpec( return new SegmentIdWithShardSpec(
dataSource, dataSource,
interval, interval,
version, version,
new NumberedShardSpec(partitionNum, 0) new NumberedShardSpec(partitionId, 0)
); );
}; };
} }
@Override @Nullable
public String getSequenceName(Interval interval, InputRow inputRow)
{
// Segments are created as needed, using a single sequence name. They may be allocated from the overlord
// (in append mode) or may be created on our own authority (in overwrite mode).
return taskId;
}
@Override @Override
public SegmentIdWithShardSpec allocate( public SegmentIdWithShardSpec allocate(
InputRow row, InputRow row,

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

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; package org.apache.druid.indexing.common.task;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.SegmentLock;
import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction; import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction;
@ -46,12 +47,20 @@ import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
* This class provides some methods to use the segment lock easier and caches the information of locked segments. * This class provides 3 functionalities.
* - {@link #verifyAndLockExistingSegments} is to verify the granularity of existing segments and lock them.
* This method must be called before the task starts indexing.
* - Tells the task what {@link LockGranularity} it should use. Note that the LockGranularity is determined in
* {@link AbstractBatchIndexTask#determineLockGranularityandTryLock}.
* - Provides some util methods for {@link LockGranularity#SEGMENT}. Also caches the information of locked segments when
* - the SEGMENt lock granularity is used.
*/ */
public class SegmentLockHelper public class TaskLockHelper
{ {
private final Map<Interval, OverwritingRootGenerationPartitions> overwritingRootGenPartitions = new HashMap<>(); private final Map<Interval, OverwritingRootGenerationPartitions> overwritingRootGenPartitions = new HashMap<>();
private final Set<DataSegment> lockedExistingSegments = new HashSet<>(); private final Set<DataSegment> lockedExistingSegments = new HashSet<>();
private final boolean useSegmentLock;
@Nullable @Nullable
private Granularity knownSegmentGranularity; private Granularity knownSegmentGranularity;
@ -84,6 +93,21 @@ public class SegmentLockHelper
} }
} }
public TaskLockHelper(boolean useSegmentLock)
{
this.useSegmentLock = useSegmentLock;
}
public boolean isUseSegmentLock()
{
return useSegmentLock;
}
public LockGranularity getLockGranularityToUse()
{
return useSegmentLock ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK;
}
public boolean hasLockedExistingSegments() public boolean hasLockedExistingSegments()
{ {
return !lockedExistingSegments.isEmpty(); return !lockedExistingSegments.isEmpty();

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

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.druid.data.input.SplitHintSpec; import org.apache.druid.data.input.SplitHintSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexSpec;
@ -185,6 +186,11 @@ public class ParallelIndexTuningConfig extends IndexTuningConfig
Preconditions.checkArgument(this.maxNumConcurrentSubTasks > 0, "maxNumConcurrentSubTasks must be positive"); Preconditions.checkArgument(this.maxNumConcurrentSubTasks > 0, "maxNumConcurrentSubTasks must be positive");
Preconditions.checkArgument(this.maxNumSegmentsToMerge > 0, "maxNumSegmentsToMerge must be positive"); Preconditions.checkArgument(this.maxNumSegmentsToMerge > 0, "maxNumSegmentsToMerge must be positive");
Preconditions.checkArgument(this.totalNumMergeTasks > 0, "totalNumMergeTasks must be positive"); Preconditions.checkArgument(this.totalNumMergeTasks > 0, "totalNumMergeTasks must be positive");
if (getPartitionsSpec() != null && getPartitionsSpec() instanceof SingleDimensionPartitionsSpec) {
if (((SingleDimensionPartitionsSpec) getPartitionsSpec()).getPartitionDimension() == null) {
throw new IAE("partitionDimension must be specified");
}
}
} }
@Nullable @Nullable

View File

@ -26,22 +26,24 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.HashPartitionCachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; import org.apache.druid.indexing.common.task.SegmentAllocators;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder;
import org.apache.druid.java.util.common.Pair; import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.ShardSpecFactory; import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nonnull;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.SortedSet;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
@ -125,14 +127,18 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
} }
@Override @Override
IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
throws IOException
{ {
return new HashPartitionCachingLocalSegmentAllocator( final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
return SegmentAllocators.forNonLinearPartitioning(
toolbox, toolbox,
getId(),
supervisorTaskId,
getDataSource(), getDataSource(),
createShardSpecs() getId(),
new SupervisorTaskAccess(supervisorTaskId, taskClient),
createHashPartitionAnalysisFromPartitionsSpec(granularitySpec, partitionsSpec)
); );
} }
@ -158,17 +164,24 @@ public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask<G
); );
} }
private Map<Interval, Pair<ShardSpecFactory, Integer>> createShardSpecs() /**
* Creates shard specs based on the given configurations. The return value is a map between intervals created
* based on the segment granularity and the shard specs to be created.
* Note that the shard specs to be created is a pair of {@link PartialShardSpec} and number of segments per interval
* and filled only when {@link #isGuaranteedRollup} = true. Otherwise, the return value contains only the set of
* intervals generated based on the segment granularity.
*/
public static HashPartitionAnalysis createHashPartitionAnalysisFromPartitionsSpec(
GranularitySpec granularitySpec,
@Nonnull HashedPartitionsSpec partitionsSpec
)
{ {
GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); final SortedSet<Interval> intervals = granularitySpec.bucketIntervals().get();
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final int numBucketsPerInterval = partitionsSpec.getNumShards() == null
final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec(); ? 1
: partitionsSpec.getNumShards();
return createShardSpecWithoutInputScan( final HashPartitionAnalysis partitionAnalysis = new HashPartitionAnalysis(partitionsSpec);
granularitySpec, intervals.forEach(interval -> partitionAnalysis.updateBucket(interval, numBucketsPerInterval));
ingestionSchema.getIOConfig(), return partitionAnalysis;
tuningConfig,
partitionsSpec
);
} }
} }

View File

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

View File

@ -28,15 +28,16 @@ import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; import org.apache.druid.indexing.common.task.SegmentAllocators;
import org.apache.druid.indexing.common.task.RangePartitionCachingLocalSegmentAllocator;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.RangePartitionIndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.common.task.batch.parallel.iterator.RangePartitionIndexTaskInputRowIteratorBuilder;
import org.apache.druid.indexing.common.task.batch.partition.RangePartitionAnalysis;
import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -149,15 +150,19 @@ public class PartialRangeSegmentGenerateTask extends PartialSegmentGenerateTask<
} }
@Override @Override
IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
throws IOException
{ {
return new RangePartitionCachingLocalSegmentAllocator( final RangePartitionAnalysis partitionAnalysis = new RangePartitionAnalysis(
(SingleDimensionPartitionsSpec) ingestionSchema.getTuningConfig().getPartitionsSpec()
);
intervalToPartitions.forEach(partitionAnalysis::updateBucket);
return SegmentAllocators.forNonLinearPartitioning(
toolbox, toolbox,
getId(),
supervisorTaskId,
getDataSource(), getDataSource(),
getPartitionDimension(ingestionSchema), getId(),
intervalToPartitions new SupervisorTaskAccess(supervisorTaskId, taskClient),
partitionAnalysis
); );
} }

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.DropwizardRowIngestionMeters;
import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMeters;
import org.apache.druid.indexing.common.task.BatchAppenderators; import org.apache.druid.indexing.common.task.BatchAppenderators;
import org.apache.druid.indexing.common.task.CachingSegmentAllocator;
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator;
import org.apache.druid.indexing.common.task.InputSourceProcessor; import org.apache.druid.indexing.common.task.InputSourceProcessor;
import org.apache.druid.indexing.common.task.NonLinearlyPartitionedSequenceNameFunction;
import org.apache.druid.indexing.common.task.SequenceNameFunction;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder;
@ -45,6 +47,7 @@ import org.apache.druid.segment.realtime.RealtimeMetricsMonitor;
import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
@ -117,16 +120,19 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
); );
final List<DataSegment> segments = generateSegments(toolbox, inputSource, tmpDir); final List<DataSegment> segments = generateSegments(toolbox, taskClient, inputSource, tmpDir);
taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments)); taskClient.report(supervisorTaskId, createGeneratedPartitionsReport(toolbox, segments));
return TaskStatus.success(getId()); return TaskStatus.success(getId());
} }
/** /**
* @return {@link IndexTaskSegmentAllocator} suitable for the desired segment partitioning strategy. * @return {@link SegmentAllocator} suitable for the desired segment partitioning strategy.
*/ */
abstract IndexTaskSegmentAllocator createSegmentAllocator(TaskToolbox toolbox) throws IOException; abstract CachingSegmentAllocator createSegmentAllocator(
TaskToolbox toolbox,
ParallelIndexSupervisorTaskClient taskClient
) throws IOException;
/** /**
* @return {@link GeneratedPartitionsReport} suitable for the desired segment partitioning strategy. * @return {@link GeneratedPartitionsReport} suitable for the desired segment partitioning strategy.
@ -138,6 +144,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
private List<DataSegment> generateSegments( private List<DataSegment> generateSegments(
final TaskToolbox toolbox, final TaskToolbox toolbox,
final ParallelIndexSupervisorTaskClient taskClient,
final InputSource inputSource, final InputSource inputSource,
final File tmpDir final File tmpDir
) throws IOException, InterruptedException, ExecutionException, TimeoutException ) throws IOException, InterruptedException, ExecutionException, TimeoutException
@ -164,7 +171,11 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec(); final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec();
final long pushTimeout = tuningConfig.getPushTimeout(); final long pushTimeout = tuningConfig.getPushTimeout();
final IndexTaskSegmentAllocator segmentAllocator = createSegmentAllocator(toolbox); final CachingSegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient);
final SequenceNameFunction sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(
getId(),
segmentAllocator.getShardSpecs()
);
final Appenderator appenderator = BatchAppenderators.newAppenderator( final Appenderator appenderator = BatchAppenderators.newAppenderator(
getId(), getId(),
@ -195,7 +206,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
inputSource, inputSource,
inputSource.needsFormat() ? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema) : null, inputSource.needsFormat() ? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema) : null,
tmpDir, tmpDir,
segmentAllocator sequenceNameFunction
); );
return pushed.getSegments(); return pushed.getSegments();

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.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional; import com.google.common.base.Optional;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.IndexingServiceClient;
@ -32,19 +31,14 @@ import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.InputSourceReader;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
import org.apache.druid.indexing.common.actions.SurrogateAction;
import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask;
import org.apache.druid.indexing.common.task.BatchAppenderators; import org.apache.druid.indexing.common.task.BatchAppenderators;
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.SegmentLockHelper; import org.apache.druid.indexing.common.task.SegmentAllocators;
import org.apache.druid.indexing.common.task.SegmentLockHelper.OverwritingRootGenerationPartitions;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
@ -69,14 +63,10 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory;
import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -230,7 +220,7 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
); );
// Find inputSegments overshadowed by pushedSegments // Find inputSegments overshadowed by pushedSegments
final Set<DataSegment> allSegments = new HashSet<>(getSegmentLockHelper().getLockedExistingSegments()); final Set<DataSegment> allSegments = new HashSet<>(getTaskLockHelper().getLockedExistingSegments());
allSegments.addAll(pushedSegments); allSegments.addAll(pushedSegments);
final VersionedIntervalTimeline<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(allSegments); final VersionedIntervalTimeline<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(allSegments);
final Set<DataSegment> oldSegments = timeline.findFullyOvershadowed() final Set<DataSegment> oldSegments = timeline.findFullyOvershadowed()
@ -279,93 +269,6 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
} }
} }
@VisibleForTesting
SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
{
return new WrappingSegmentAllocator(toolbox, taskClient);
}
private class WrappingSegmentAllocator implements SegmentAllocator
{
private final TaskToolbox toolbox;
private final ParallelIndexSupervisorTaskClient taskClient;
private SegmentAllocator internalAllocator;
private WrappingSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient)
{
this.toolbox = toolbox;
this.taskClient = taskClient;
}
@Override
public SegmentIdWithShardSpec allocate(
InputRow row,
String sequenceName,
String previousSegmentId,
boolean skipSegmentLineageCheck
) throws IOException
{
if (internalAllocator == null) {
internalAllocator = createSegmentAllocator();
}
return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck);
}
private SegmentAllocator createSegmentAllocator()
{
final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
final SegmentLockHelper segmentLockHelper = getSegmentLockHelper();
if (ingestionSchema.getIOConfig().isAppendToExisting() || isUseSegmentLock()) {
return new ActionBasedSegmentAllocator(
toolbox.getTaskActionClient(),
ingestionSchema.getDataSchema(),
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> {
final Interval interval = granularitySpec
.bucketInterval(row.getTimestamp())
.or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp()));
final ShardSpecFactory shardSpecFactory;
if (segmentLockHelper.hasOverwritingRootGenerationPartition(interval)) {
final OverwritingRootGenerationPartitions overwritingSegmentMeta = segmentLockHelper
.getOverwritingRootGenerationPartition(interval);
if (overwritingSegmentMeta == null) {
throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval);
}
shardSpecFactory = new NumberedOverwritingShardSpecFactory(
overwritingSegmentMeta.getStartRootPartitionId(),
overwritingSegmentMeta.getEndRootPartitionId(),
overwritingSegmentMeta.getMinorVersionForNewSegments()
);
} else {
shardSpecFactory = NumberedShardSpecFactory.instance();
}
return new SurrogateAction<>(
supervisorTaskId,
new SegmentAllocateAction(
schema.getDataSource(),
row.getTimestamp(),
schema.getGranularitySpec().getQueryGranularity(),
schema.getGranularitySpec().getSegmentGranularity(),
sequenceName,
previousSegmentId,
skipSegmentLineageCheck,
shardSpecFactory,
isUseSegmentLock() ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK
)
);
}
);
} else {
return (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> taskClient.allocateSegment(
supervisorTaskId,
row.getTimestamp()
);
}
}
}
/** /**
* This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}. * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
* If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs * If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs
@ -410,7 +313,14 @@ public class SinglePhaseSubTask extends AbstractBatchIndexTask
final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec(); final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec();
final long pushTimeout = tuningConfig.getPushTimeout(); final long pushTimeout = tuningConfig.getPushTimeout();
final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent();
final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning(
toolbox,
new SupervisorTaskAccess(getSupervisorTaskId(), taskClient),
getIngestionSchema().getDataSchema(),
getTaskLockHelper(),
ingestionSchema.getIOConfig().isAppendToExisting(),
partitionsSpec
);
final Appenderator appenderator = BatchAppenderators.newAppenderator( final Appenderator appenderator = BatchAppenderators.newAppenderator(
getId(), getId(),

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

View File

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

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

View File

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

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.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
import org.apache.druid.utils.CircularBuffer; import org.apache.druid.utils.CircularBuffer;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -232,7 +232,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
sequenceName, sequenceName,
previousSegmentId, previousSegmentId,
skipSegmentLineageCheck, skipSegmentLineageCheck,
NumberedShardSpecFactory.instance(), NumberedPartialShardSpec.instance(),
lockGranularityToUse lockGranularityToUse
) )
), ),

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.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.HashBasedNumberedPartialShardSpec;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.LinearPartialShardSpec;
import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.LinearShardSpec;
import org.apache.druid.timeline.partition.LinearShardSpecFactory; import org.apache.druid.timeline.partition.NumberedPartialShardSpec;
import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.timeline.partition.PartialShardSpec;
import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.DateTime; import org.joda.time.DateTime;
@ -652,7 +652,7 @@ public class SegmentAllocateActionTest
Granularities.HOUR, Granularities.HOUR,
"s1", "s1",
null, null,
LinearShardSpecFactory.instance() LinearPartialShardSpec.instance()
); );
final SegmentIdWithShardSpec id2 = allocate( final SegmentIdWithShardSpec id2 = allocate(
task, task,
@ -661,7 +661,7 @@ public class SegmentAllocateActionTest
Granularities.HOUR, Granularities.HOUR,
"s1", "s1",
id1.toString(), id1.toString(),
LinearShardSpecFactory.instance() LinearPartialShardSpec.instance()
); );
assertSameIdentifier( assertSameIdentifier(
@ -899,39 +899,7 @@ public class SegmentAllocateActionTest
} }
@Test @Test
public void testSerde() throws Exception public void testWithPartialShardSpecAndOvershadowingSegments() throws IOException
{
final ObjectMapper objectMapper = new DefaultObjectMapper();
objectMapper.registerSubtypes(NumberedShardSpecFactory.class);
final SegmentAllocateAction action = new SegmentAllocateAction(
DATA_SOURCE,
PARTY_TIME,
Granularities.MINUTE,
Granularities.HOUR,
"s1",
"prev",
false,
NumberedShardSpecFactory.instance(),
lockGranularity
);
final SegmentAllocateAction action2 = (SegmentAllocateAction) objectMapper.readValue(
objectMapper.writeValueAsBytes(action),
TaskAction.class
);
Assert.assertEquals(action.getDataSource(), action2.getDataSource());
Assert.assertEquals(action.getTimestamp(), action2.getTimestamp());
Assert.assertEquals(action.getQueryGranularity(), action2.getQueryGranularity());
Assert.assertEquals(action.getPreferredSegmentGranularity(), action2.getPreferredSegmentGranularity());
Assert.assertEquals(action.getSequenceName(), action2.getSequenceName());
Assert.assertEquals(action.getPreviousSegmentId(), action2.getPreviousSegmentId());
Assert.assertEquals(action.isSkipSegmentLineageCheck(), action2.isSkipSegmentLineageCheck());
}
@Test
public void testWithShardSpecFactoryAndOvershadowingSegments() throws IOException
{ {
final Task task = NoopTask.create(); final Task task = NoopTask.create();
taskActionTestKit.getTaskLockbox().add(task); taskActionTestKit.getTaskLockbox().add(task);
@ -965,7 +933,7 @@ public class SegmentAllocateActionTest
"seq", "seq",
null, null,
true, true,
new HashBasedNumberedShardSpecFactory(ImmutableList.of("dim1"), 2), new HashBasedNumberedPartialShardSpec(ImmutableList.of("dim1"), 2),
lockGranularity lockGranularity
); );
final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox()); final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox());
@ -996,7 +964,7 @@ public class SegmentAllocateActionTest
preferredSegmentGranularity, preferredSegmentGranularity,
sequenceName, sequenceName,
sequencePreviousId, sequencePreviousId,
NumberedShardSpecFactory.instance() NumberedPartialShardSpec.instance()
); );
} }
@ -1007,7 +975,7 @@ public class SegmentAllocateActionTest
final Granularity preferredSegmentGranularity, final Granularity preferredSegmentGranularity,
final String sequenceName, final String sequenceName,
final String sequencePreviousId, final String sequencePreviousId,
final ShardSpecFactory shardSpecFactory final PartialShardSpec partialShardSpec
) )
{ {
final SegmentAllocateAction action = new SegmentAllocateAction( final SegmentAllocateAction action = new SegmentAllocateAction(
@ -1018,7 +986,7 @@ public class SegmentAllocateActionTest
sequenceName, sequenceName,
sequencePreviousId, sequencePreviousId,
false, false,
shardSpecFactory, partialShardSpec,
lockGranularity lockGranularity
); );
return action.perform(task, taskActionTestKit.getTaskActionToolbox()); return action.perform(task, taskActionTestKit.getTaskActionToolbox());

View File

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

View File

@ -386,7 +386,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
Granularities.MINUTE, Granularities.MINUTE,
null null
), ),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true), IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
false false
), ),
null, null,
@ -726,7 +726,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
), ),
false false
), ),
IndexTaskTest.createTuningConfig(5000000, null, null, Long.MAX_VALUE, null, null, false, true) IndexTaskTest.createTuningConfig(5000000, null, null, Long.MAX_VALUE, null, false, true)
), ),
null, null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
@ -796,7 +796,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
Granularities.MINUTE, Granularities.MINUTE,
null null
), ),
IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true), IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, false, true),
appendToExisting appendToExisting
), ),
null, null,

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.TaskStatus;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskReport;
@ -459,7 +461,7 @@ public class IndexTaskTest extends IngestionTestBase
null, null,
null, null,
null, null,
createTuningConfigWithNumShards(1, null, true), createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 1, null), true),
false false
), ),
null, null,
@ -501,7 +503,7 @@ public class IndexTaskTest extends IngestionTestBase
null, null,
null, null,
null, null,
createTuningConfigWithNumShards(2, ImmutableList.of("dim"), true), createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")), true),
false false
), ),
null, null,
@ -558,7 +560,7 @@ public class IndexTaskTest extends IngestionTestBase
} }
@Test @Test
public void testAppendToExisting() throws Exception public void testWriteNewSegmentsWithAppendToExistingWithLinearPartitioningSuccessfullyAppend() throws Exception
{ {
File tmpDir = temporaryFolder.newFolder(); File tmpDir = temporaryFolder.newFolder();
File tmpFile = File.createTempFile("druid", "index", tmpDir); File tmpFile = File.createTempFile("druid", "index", tmpDir);
@ -787,7 +789,7 @@ public class IndexTaskTest extends IngestionTestBase
Granularities.MINUTE, Granularities.MINUTE,
null null
), ),
createTuningConfig(2, 2, null, 2L, null, null, false, true), createTuningConfig(2, 2, null, 2L, null, false, true),
false false
), ),
null, null,
@ -835,7 +837,7 @@ public class IndexTaskTest extends IngestionTestBase
true, true,
null null
), ),
createTuningConfig(3, 2, null, 2L, null, null, true, true), createTuningConfig(3, 2, null, 2L, null, true, true),
false false
), ),
null, null,
@ -882,7 +884,7 @@ public class IndexTaskTest extends IngestionTestBase
true, true,
null null
), ),
createTuningConfig(3, 2, null, 2L, null, null, false, true), createTuningConfig(3, 2, null, 2L, null, false, true),
false false
), ),
null, null,
@ -950,7 +952,7 @@ public class IndexTaskTest extends IngestionTestBase
0 0
), ),
null, null,
createTuningConfig(2, null, null, null, null, null, false, false), // ignore parse exception, createTuningConfig(2, null, null, null, null, false, false), // ignore parse exception,
false false
); );
@ -998,7 +1000,7 @@ public class IndexTaskTest extends IngestionTestBase
0 0
), ),
null, null,
createTuningConfig(2, null, null, null, null, null, false, true), // report parse exception createTuningConfig(2, null, null, null, null, false, true), // report parse exception
false false
); );
@ -1418,7 +1420,7 @@ public class IndexTaskTest extends IngestionTestBase
0 0
), ),
null, null,
createTuningConfig(2, 1, null, null, null, null, true, true), // report parse exception createTuningConfig(2, 1, null, null, null, true, true), // report parse exception
false false
); );
@ -1483,7 +1485,7 @@ public class IndexTaskTest extends IngestionTestBase
0 0
), ),
null, null,
createTuningConfig(2, null, null, null, null, null, false, true), // report parse exception createTuningConfig(2, null, null, null, null, false, true), // report parse exception
false false
); );
@ -1538,7 +1540,7 @@ public class IndexTaskTest extends IngestionTestBase
true, true,
null null
), ),
createTuningConfig(3, 2, null, 2L, null, null, false, true), createTuningConfig(3, 2, null, 2L, null, false, true),
false false
), ),
null, null,
@ -1607,7 +1609,7 @@ public class IndexTaskTest extends IngestionTestBase
true, true,
null null
), ),
createTuningConfig(3, 2, null, 2L, null, null, false, true), createTuningConfig(3, 2, null, 2L, null, false, true),
false false
), ),
null, null,
@ -1634,6 +1636,35 @@ public class IndexTaskTest extends IngestionTestBase
} }
} }
@Test
public void testIndexTaskWitSingleDimPartitionsSpecThrowingException() throws Exception
{
final IndexTask task = new IndexTask(
null,
null,
createIngestionSpec(
useInputFormatApi,
jsonMapper,
null,
null,
null,
null,
createTuningConfigWithPartitionsSpec(new SingleDimensionPartitionsSpec(null, 1, null, false), true),
false
),
null,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
null,
rowIngestionMetersFactory,
appenderatorsManager
);
expectedException.expect(UnsupportedOperationException.class);
expectedException.expectMessage(
"partitionsSpec[org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec] is not supported"
);
task.isReady(createActionClient(task));
}
public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus status) public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus status)
{ {
// full stacktrace will be too long and make tests brittle (e.g. if line # changes), just match the main message // full stacktrace will be too long and make tests brittle (e.g. if line # changes), just match the main message
@ -1662,15 +1693,13 @@ public class IndexTaskTest extends IngestionTestBase
null, null,
null, null,
null, null,
null,
forceGuaranteedRollup, forceGuaranteedRollup,
true true
); );
} }
private static IndexTuningConfig createTuningConfigWithNumShards( private static IndexTuningConfig createTuningConfigWithPartitionsSpec(
int numShards, PartitionsSpec partitionsSpec,
@Nullable List<String> partitionDimensions,
boolean forceGuaranteedRollup boolean forceGuaranteedRollup
) )
{ {
@ -1679,8 +1708,7 @@ public class IndexTaskTest extends IngestionTestBase
1, 1,
null, null,
null, null,
numShards, partitionsSpec,
partitionDimensions,
forceGuaranteedRollup, forceGuaranteedRollup,
true true
); );
@ -1691,8 +1719,7 @@ public class IndexTaskTest extends IngestionTestBase
@Nullable Integer maxRowsInMemory, @Nullable Integer maxRowsInMemory,
@Nullable Long maxBytesInMemory, @Nullable Long maxBytesInMemory,
@Nullable Long maxTotalRows, @Nullable Long maxTotalRows,
@Nullable Integer numShards, @Nullable PartitionsSpec partitionsSpec,
@Nullable List<String> partitionDimensions,
boolean forceGuaranteedRollup, boolean forceGuaranteedRollup,
boolean reportParseException boolean reportParseException
) )
@ -1704,9 +1731,9 @@ public class IndexTaskTest extends IngestionTestBase
maxBytesInMemory, maxBytesInMemory,
maxTotalRows, maxTotalRows,
null, null,
numShards,
partitionDimensions,
null, null,
null,
partitionsSpec,
INDEX_SPEC, INDEX_SPEC,
null, null,
null, null,

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

View File

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

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

View File

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

View File

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

View File

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

View File

@ -20,15 +20,23 @@
package org.apache.druid.indexing.common.task.batch.parallel; package org.apache.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.io.File; import java.io.File;
import java.util.List;
public class PartialHashSegmentGenerateTaskTest public class PartialHashSegmentGenerateTaskTest
{ {
@ -71,4 +79,28 @@ public class PartialHashSegmentGenerateTaskTest
String id = target.getId(); String id = target.getId();
Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentGenerateTask.TYPE)); Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentGenerateTask.TYPE));
} }
@Test
public void testCreateHashPartitionAnalysisFromPartitionsSpecWithNumShardsReturningAnalysisOfValidNumBuckets()
{
final List<Interval> intervals = ImmutableList.of(
Intervals.of("2020-01-01/2020-01-02"),
Intervals.of("2020-01-02/2020-01-03"),
Intervals.of("2020-01-03/2020-01-04")
);
final int expectedNumBuckets = 5;
final HashPartitionAnalysis partitionAnalysis = PartialHashSegmentGenerateTask
.createHashPartitionAnalysisFromPartitionsSpec(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
intervals
),
new HashedPartitionsSpec(null, expectedNumBuckets, null)
);
Assert.assertEquals(intervals.size(), partitionAnalysis.getNumTimePartitions());
for (Interval interval : intervals) {
Assert.assertEquals(expectedNumBuckets, partitionAnalysis.getBucketAnalysis(interval).intValue());
}
}
} }

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.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory;
import org.apache.druid.indexing.common.task.batch.parallel.distribution.PartitionBoundaries;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.timeline.partition.PartitionBoundaries;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;

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

View File

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

View File

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

View File

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

View File

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

View File

@ -22,7 +22,7 @@ package org.apache.druid.indexing.overlord;
import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Pair;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.ShardSpecFactory; import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -157,7 +157,7 @@ public interface IndexerMetadataStorageCoordinator
* @param previousSegmentId previous segment in the series; may be null or empty, meaning this is the first * @param previousSegmentId previous segment in the series; may be null or empty, meaning this is the first
* segment * segment
* @param interval interval for which to allocate a segment * @param interval interval for which to allocate a segment
* @param shardSpecFactory shardSpecFactory containing all necessary information to create a shardSpec for the * @param partialShardSpec partialShardSpec containing all necessary information to create a shardSpec for the
* new segmentId * new segmentId
* @param maxVersion use this version if we have no better version to use. The returned segment * @param maxVersion use this version if we have no better version to use. The returned segment
* identifier may have a version lower than this one, but will not have one higher. * identifier may have a version lower than this one, but will not have one higher.
@ -171,7 +171,7 @@ public interface IndexerMetadataStorageCoordinator
String sequenceName, String sequenceName,
@Nullable String previousSegmentId, @Nullable String previousSegmentId,
Interval interval, Interval interval,
ShardSpecFactory shardSpecFactory, PartialShardSpec partialShardSpec,
String maxVersion, String maxVersion,
boolean skipSegmentLineageCheck boolean skipSegmentLineageCheck
); );

View File

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

View File

@ -21,25 +21,32 @@ package org.apache.druid.segment.realtime.appenderator;
import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRow;
import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
public interface SegmentAllocator public interface SegmentAllocator
{ {
/** /**
* Allocates a new segment for a given timestamp. * Allocates a new segment for a given timestamp. Even though its name is "allocate", this method is actually
* idempotent: given the same sequenceName, previousSegmentId, and skipSegmentLineageCheck, the implementation
* must return the same segment ID.
* *
* @param row the event which triggered this allocation request * @param row the event which triggered this allocation request
* @param sequenceName sequenceName for this allocation * @param sequenceName sequenceName for this allocation
* @param previousSegmentId segment identifier returned on the previous call to allocate for your sequenceName * @param previousSegmentId segment identifier returned on the previous call to allocate for your sequenceName.
* When skipSegmentLineageCheck is false, this can be null if it is the first call
* for the same sequenceName.
* When skipSegmentLineageCheck is true, this will be ignored.
* @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence. * @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence.
* Should be set to false if replica tasks would index events in same order * Should be set to false if replica tasks would index events in same order
* *
* @return the pending segment identifier, or null if it was impossible to allocate a new segment * @return the pending segment identifier, or null if it was impossible to allocate a new segment
*/ */
@Nullable
SegmentIdWithShardSpec allocate( SegmentIdWithShardSpec allocate(
InputRow row, InputRow row,
String sequenceName, String sequenceName,
String previousSegmentId, @Nullable String previousSegmentId,
boolean skipSegmentLineageCheck boolean skipSegmentLineageCheck
) throws IOException; ) throws IOException;
} }

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

View File

@ -36,6 +36,8 @@ import org.junit.Test;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class HashBasedNumberedShardSpecTest public class HashBasedNumberedShardSpecTest
{ {
@ -126,7 +128,6 @@ public class HashBasedNumberedShardSpecTest
@Test @Test
public void testIsInChunk() public void testIsInChunk()
{ {
List<ShardSpec> specs = new ArrayList<>(); List<ShardSpec> specs = new ArrayList<>();
for (int i = 0; i < 3; i++) { for (int i = 0; i < 3; i++) {
specs.add(new HashOverridenShardSpec(i, 3)); specs.add(new HashOverridenShardSpec(i, 3));
@ -137,7 +138,20 @@ public class HashBasedNumberedShardSpecTest
assertExistsInOneSpec(specs, new HashInputRow(0)); assertExistsInOneSpec(specs, new HashInputRow(0));
assertExistsInOneSpec(specs, new HashInputRow(1000)); assertExistsInOneSpec(specs, new HashInputRow(1000));
assertExistsInOneSpec(specs, new HashInputRow(-1000)); assertExistsInOneSpec(specs, new HashInputRow(-1000));
}
@Test
public void testIsInChunkWithMorePartitionsBeyondNumBucketsReturningTrue()
{
final int numBuckets = 3;
final List<ShardSpec> specs = IntStream.range(0, 10)
.mapToObj(i -> new HashOverridenShardSpec(i, numBuckets))
.collect(Collectors.toList());
for (int i = 0; i < 10; i++) {
final InputRow row = new HashInputRow(numBuckets * 10000 + i);
Assert.assertTrue(specs.get(i).isInChunk(row.getTimestampFromEpoch(), row));
}
} }
@Test @Test
@ -186,10 +200,7 @@ public class HashBasedNumberedShardSpecTest
public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec
{ {
public HashOverridenShardSpec( public HashOverridenShardSpec(int partitionNum, int partitions)
int partitionNum,
int partitions
)
{ {
super(partitionNum, partitions, null, ServerTestHelper.MAPPER); super(partitionNum, partitions, null, ServerTestHelper.MAPPER);
} }