You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ji...@apache.org on 2019/07/25 00:35:57 UTC
[incubator-druid] branch master updated: Add support minor
compaction with segment locking (#7547)
This is an automated email from the ASF dual-hosted git repository.
jihoonson pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git
The following commit(s) were added to refs/heads/master by this push:
new db14946 Add support minor compaction with segment locking (#7547)
db14946 is described below
commit db149462073d59e7563f0d3834e69d44a2bb4011
Author: Jihoon Son <ji...@apache.org>
AuthorDate: Wed Jul 24 17:35:46 2019 -0700
Add support minor compaction with segment locking (#7547)
* Segment locking
* Allow both timeChunk and segment lock in the same gruop
* fix it test
* Fix adding same chunk to atomicUpdateGroup
* resolving todos
* Fix segments to lock
* fix segments to lock
* fix kill task
* resolving todos
* resolving todos
* fix teamcity
* remove unused class
* fix single map
* resolving todos
* fix build
* fix SQLMetadataSegmentManager
* fix findInputSegments
* adding more tests
* fixing task lock checks
* add SegmentTransactionalOverwriteAction
* changing publisher
* fixing something
* fix for perfect rollup
* fix test
* adjust package-lock.json
* fix test
* fix style
* adding javadocs
* remove unused classes
* add more javadocs
* unused import
* fix test
* fix test
* Support forceTimeChunk context and force timeChunk lock for parallel index task if intervals are missing
* fix travis
* fix travis
* unused import
* spotbug
* revert getMaxVersion
* address comments
* fix tc
* add missing error handling
* fix backward compatibility
* unused import
* Fix perf of versionedIntervalTimeline
* fix timeline
* fix tc
* remove remaining todos
* add comment for parallel index
* fix javadoc and typos
* typo
* address comments
---
.../util/common/granularity/AllGranularity.java | 9 +
.../common/granularity/DurationGranularity.java | 10 +
.../java/util/common/granularity/Granularity.java | 5 +
.../util/common/granularity/NoneGranularity.java | 10 +
.../util/common/granularity/PeriodGranularity.java | 7 +
.../org/apache/druid/timeline/DataSegment.java | 91 +-
.../org/apache/druid/timeline/Overshadowable.java | 78 ++
.../java/org/apache/druid/timeline/SegmentId.java | 1 -
.../org/apache/druid/timeline/TimelineLookup.java | 2 +-
.../druid/timeline/TimelineObjectHolder.java | 25 +-
.../druid/timeline/VersionedIntervalTimeline.java | 151 +++-
.../timeline/partition/AtomicUpdateGroup.java | 175 ++++
.../partition/HashBasedNumberedShardSpec.java | 10 +-
.../HashBasedNumberedShardSpecFactory.java | 101 +++
.../partition/ImmutablePartitionHolder.java | 4 +-
.../timeline/partition/IntegerPartitionChunk.java | 6 +-
.../druid/timeline/partition/LinearShardSpec.java | 6 +
...tionHolder.java => LinearShardSpecFactory.java} | 34 +-
.../druid/timeline/partition/NoneShardSpec.java | 8 +-
.../partition/NumberedOverwriteShardSpec.java | 229 +++++
.../NumberedOverwritingPartitionChunk.java | 118 +++
.../NumberedOverwritingShardSpecFactory.java | 90 ++
.../timeline/partition/NumberedPartitionChunk.java | 10 +
.../timeline/partition/NumberedShardSpec.java | 6 +
.../partition/NumberedShardSpecFactory.java | 61 ++
.../timeline/partition/OvershadowableManager.java | 805 +++++++++++++++++
...artitionHolder.java => OverwriteShardSpec.java} | 23 +-
.../druid/timeline/partition/PartitionHolder.java | 66 +-
...tablePartitionHolder.java => PartitionIds.java} | 34 +-
.../apache/druid/timeline/partition/ShardSpec.java | 28 +-
.../druid/timeline/partition/ShardSpecFactory.java | 56 ++
.../partition/SingleDimensionShardSpec.java | 17 +-
.../partition/SingleElementPartitionChunk.java | 2 +-
.../timeline/partition/StringPartitionChunk.java | 11 +
.../org/apache/druid/timeline/DataSegmentTest.java | 11 +-
.../timeline/VersionedIntervalTimelineTest.java | 958 +++++++++++++++------
.../partition/IntegerPartitionChunkTest.java | 103 ++-
.../IncrementalPublishingKafkaIndexTaskRunner.java | 7 +-
.../druid/indexing/kafka/KafkaIndexTask.java | 3 +-
.../druid/indexing/kafka/KafkaIndexTaskTest.java | 393 +++++----
.../druid/indexing/kinesis/KinesisIndexTask.java | 4 +-
.../indexing/kinesis/KinesisIndexTaskRunner.java | 7 +-
.../indexing/kinesis/KinesisIndexTaskTest.java | 366 ++++----
.../apache/druid/indexer/IndexGeneratorJob.java | 1 +
.../{TaskLockType.java => LockGranularity.java} | 9 +-
.../apache/druid/indexing/common/SegmentLock.java | 233 +++++
.../org/apache/druid/indexing/common/TaskLock.java | 166 +---
.../apache/druid/indexing/common/TaskLockType.java | 2 +-
.../common/{TaskLock.java => TimeChunkLock.java} | 120 ++-
.../common/actions/LocalTaskActionClient.java | 18 +-
.../common/actions/SegmentAllocateAction.java | 102 ++-
.../common/actions/SegmentInsertAction.java | 2 +-
.../common/actions/SegmentListUsedAction.java | 3 +-
...reAction.java => SegmentLockAcquireAction.java} | 77 +-
...reAction.java => SegmentLockReleaseAction.java} | 50 +-
...ction.java => SegmentLockTryAcquireAction.java} | 71 +-
.../actions/SegmentMetadataUpdateAction.java | 8 +-
.../indexing/common/actions/SegmentNukeAction.java | 8 +-
.../actions/SegmentTransactionalInsertAction.java | 133 ++-
.../druid/indexing/common/actions/TaskAction.java | 6 +-
.../common/actions/TaskActionPreconditions.java | 94 --
.../druid/indexing/common/actions/TaskLocks.java | 162 ++++
...Action.java => TimeChunkLockAcquireAction.java} | 20 +-
...ion.java => TimeChunkLockTryAcquireAction.java} | 18 +-
.../indexing/common/index/YeOldePlumberSchool.java | 3 +-
.../common/task/AbstractBatchIndexTask.java | 384 +++++++++
.../common/task/AbstractFixedIntervalTask.java | 4 +-
.../druid/indexing/common/task/AbstractTask.java | 2 +-
.../task/AppenderatorDriverRealtimeIndexTask.java | 56 +-
.../common/task/CachingLocalSegmentAllocator.java | 179 ++++
.../druid/indexing/common/task/CompactionTask.java | 95 +-
.../indexing/common/task/HadoopIndexTask.java | 42 +-
.../druid/indexing/common/task/HadoopTask.java | 2 +-
.../druid/indexing/common/task/IndexTask.java | 352 ++++----
.../common/task/IndexTaskSegmentAllocator.java | 42 +
.../druid/indexing/common/task/KillTask.java | 58 +-
.../common/task/LocalSegmentAllocator.java | 104 +++
.../druid/indexing/common/task/NoopTask.java | 16 +-
.../indexing/common/task/RealtimeIndexTask.java | 8 +-
.../common/task/RemoteSegmentAllocator.java | 127 +++
.../indexing/common/task/SegmentLockHelper.java | 291 +++++++
.../apache/druid/indexing/common/task/Tasks.java | 24 +-
.../task/batch/parallel/ParallelIndexSubTask.java | 238 +++--
.../parallel/ParallelIndexSupervisorTask.java | 112 ++-
.../batch/parallel/ParallelIndexTaskClient.java | 6 +-
.../task/batch/parallel/PushedSegmentsReport.java | 21 +-
.../SinglePhaseParallelIndexTaskRunner.java | 42 +-
.../firehose/IngestSegmentFirehoseFactory.java | 2 +
.../druid/indexing/overlord/CriticalAction.java | 4 +-
.../druid/indexing/overlord/LockRequest.java | 44 +-
.../overlord/LockRequestForNewSegment.java | 197 +++++
.../apache/druid/indexing/overlord/LockResult.java | 27 +-
.../overlord/SpecificSegmentLockRequest.java | 172 ++++
.../druid/indexing/overlord/TaskLockbox.java | 701 +++++++--------
.../indexing/overlord/TimeChunkLockRequest.java | 160 ++++
.../seekablestream/SeekableStreamIndexTask.java | 10 +-
.../SeekableStreamIndexTaskRunner.java | 39 +-
.../indexing/seekablestream/SequenceMetadata.java | 11 +-
.../apache/druid/indexing/common/TaskLockTest.java | 197 +++++
.../common/actions/RemoteTaskActionClientTest.java | 7 +-
.../common/actions/SegmentAllocateActionTest.java | 691 ++++++++++-----
.../common/actions/SegmentInsertActionTest.java | 16 +-
.../SegmentTransactionalInsertActionTest.java | 31 +-
.../common/actions/SurrogateActionTest.java | 5 +-
.../actions/TaskActionPreconditionsTest.java | 150 ----
.../indexing/common/actions/TaskActionTestKit.java | 2 +-
.../indexing/common/actions/TaskLocksTest.java | 324 +++++++
...st.java => TimeChunkLockAcquireActionTest.java} | 14 +-
...java => TimeChunkLockTryAcquireActionTest.java} | 14 +-
.../AppenderatorDriverRealtimeIndexTaskTest.java | 5 +-
.../common/task/CompactionTaskRunTest.java | 419 ++++++++-
.../indexing/common/task/CompactionTaskTest.java | 4 +-
.../druid/indexing/common/task/HadoopTaskTest.java | 31 +
.../druid/indexing/common/task/IndexTaskTest.java | 460 +++++-----
.../indexing/common/task/IngestionTestBase.java | 298 +++++--
.../druid/indexing/common/task/KillTaskTest.java | 107 +++
.../common/task/RealtimeIndexTaskTest.java | 2 +-
.../AbstractParallelIndexSupervisorTaskTest.java | 21 +-
.../ParallelIndexSupervisorTaskResourceTest.java | 62 +-
.../parallel/ParallelIndexSupervisorTaskTest.java | 137 ++-
.../task/batch/parallel/TaskMonitorTest.java | 2 +-
.../firehose/IngestSegmentFirehoseFactoryTest.java | 47 +-
.../druid/indexing/overlord/RealtimeishTask.java | 6 +-
.../overlord/SingleTaskBackgroundRunnerTest.java | 4 +-
.../druid/indexing/overlord/TaskLifecycleTest.java | 20 +-
.../overlord/TaskLockBoxConcurrencyTest.java | 29 +-
.../druid/indexing/overlord/TaskLockboxTest.java | 567 ++++++++++--
.../druid/indexing/overlord/http/OverlordTest.java | 4 +-
.../OverlordSecurityResourceFilterTest.java | 2 +-
...butionWithAffinityWorkerSelectStrategyTest.java | 6 +-
.../EqualDistributionWorkerSelectStrategyTest.java | 10 +-
...pacityWithAffinityWorkerSelectStrategyTest.java | 6 +-
.../TestIndexerMetadataStorageCoordinator.java | 12 +-
.../indexing/worker/WorkerTaskManagerTest.java | 2 +-
.../clients/CoordinatorResourceTestClient.java | 8 +-
.../tests/indexer/AbstractITBatchIndexTest.java | 21 +-
.../druid/segment/ReferenceCountingSegment.java | 81 +-
.../apache/druid/query/QueryRunnerTestHelper.java | 5 +-
.../query/select/MultiSegmentSelectQueryTest.java | 10 +-
.../timeboundary/TimeBoundaryQueryRunnerTest.java | 8 +-
.../apache/druid/segment/SchemalessIndexTest.java | 63 +-
.../apache/druid/client/DataSourcesSnapshot.java | 4 +-
.../org/apache/druid/client/SegmentLoadInfo.java | 39 +-
.../druid/client/selector/ServerSelector.java | 41 +-
.../IndexerMetadataStorageCoordinator.java | 11 +-
.../IndexerSQLMetadataStorageCoordinator.java | 149 ++--
.../druid/metadata/SQLMetadataSegmentManager.java | 2 +-
.../AppenderatorDriverSegmentLockHelper.java | 12 +-
.../realtime/appenderator/AppenderatorImpl.java | 11 +-
.../realtime/appenderator/Appenderators.java | 8 -
.../appenderator/BaseAppenderatorDriver.java | 4 +-
.../appenderator/BatchAppenderatorDriver.java | 29 +-
.../appenderator/SegmentIdWithShardSpec.java | 15 +
.../appenderator/StreamAppenderatorDriver.java | 11 +-
.../TransactionalSegmentPublisher.java | 77 +-
.../segment/realtime/plumber/RealtimePlumber.java | 2 +-
.../druid/segment/realtime/plumber/Sink.java | 46 +-
.../org/apache/druid/server/SegmentManager.java | 22 +-
.../DruidCoordinatorCleanupOvershadowed.java | 3 +-
.../druid/client/CachingClusteredClientTest.java | 85 +-
.../IndexerSQLMetadataStorageCoordinatorTest.java | 183 +++-
.../appenderator/BatchAppenderatorDriverTest.java | 14 +-
.../StreamAppenderatorDriverFailTest.java | 16 +-
.../appenderator/StreamAppenderatorDriverTest.java | 15 +-
.../helper/NewestSegmentFirstPolicyTest.java | 2 +-
.../druid/server/shard/NumberedShardSpecTest.java | 131 ++-
.../partition/HashBasedNumberedShardSpecTest.java | 1 -
.../druid/sql/calcite/schema/SystemSchemaTest.java | 24 +-
.../util/SpecificSegmentsQuerySegmentWalker.java | 21 +-
169 files changed, 10740 insertions(+), 3167 deletions(-)
diff --git a/core/src/main/java/org/apache/druid/java/util/common/granularity/AllGranularity.java b/core/src/main/java/org/apache/druid/java/util/common/granularity/AllGranularity.java
index 7299bf5..01ed140 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/granularity/AllGranularity.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/granularity/AllGranularity.java
@@ -62,6 +62,15 @@ public class AllGranularity extends Granularity
throw new UnsupportedOperationException("This method should not be invoked for this granularity type");
}
+ /**
+ * No interval is aligned with all granularity since it's infinite.
+ */
+ @Override
+ public boolean isAligned(Interval interval)
+ {
+ return false;
+ }
+
@Override
public byte[] getCacheKey()
{
diff --git a/core/src/main/java/org/apache/druid/java/util/common/granularity/DurationGranularity.java b/core/src/main/java/org/apache/druid/java/util/common/granularity/DurationGranularity.java
index a48c071..d9af7ed 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/granularity/DurationGranularity.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/granularity/DurationGranularity.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.DateTimes;
import org.joda.time.DateTime;
+import org.joda.time.Interval;
import org.joda.time.format.DateTimeFormatter;
import java.nio.ByteBuffer;
@@ -95,6 +96,15 @@ public class DurationGranularity extends Granularity
}
@Override
+ public boolean isAligned(Interval interval)
+ {
+ if (interval.toDurationMillis() == duration) {
+ return (interval.getStartMillis() - origin) % duration == 0;
+ }
+ return false;
+ }
+
+ @Override
public byte[] getCacheKey()
{
return ByteBuffer.allocate(2 * Long.BYTES).putLong(duration).putLong(origin).array();
diff --git a/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java b/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java
index e2b7c33..3a9cdac 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/granularity/Granularity.java
@@ -113,6 +113,11 @@ public abstract class Granularity implements Cacheable
public abstract DateTime toDate(String filePath, Formatter formatter);
+ /**
+ * Return true if time chunks populated by this granularity includes the given interval time chunk.
+ */
+ public abstract boolean isAligned(Interval interval);
+
public DateTime bucketEnd(DateTime time)
{
return increment(bucketStart(time));
diff --git a/core/src/main/java/org/apache/druid/java/util/common/granularity/NoneGranularity.java b/core/src/main/java/org/apache/druid/java/util/common/granularity/NoneGranularity.java
index 06c65b0..750ea2a 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/granularity/NoneGranularity.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/granularity/NoneGranularity.java
@@ -20,6 +20,7 @@
package org.apache.druid.java.util.common.granularity;
import org.joda.time.DateTime;
+import org.joda.time.Interval;
import org.joda.time.format.DateTimeFormatter;
/**
@@ -59,6 +60,15 @@ public class NoneGranularity extends Granularity
throw new UnsupportedOperationException("This method should not be invoked for this granularity type");
}
+ /**
+ * Any interval is aligned with none granularity since it's effectively millisecond granularity.
+ */
+ @Override
+ public boolean isAligned(Interval interval)
+ {
+ return true;
+ }
+
@Override
public byte[] getCacheKey()
{
diff --git a/core/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java b/core/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java
index a88d530..b1f36e6 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/granularity/PeriodGranularity.java
@@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.StringUtils;
import org.joda.time.Chronology;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
+import org.joda.time.Interval;
import org.joda.time.Period;
import org.joda.time.chrono.ISOChronology;
import org.joda.time.format.DateTimeFormat;
@@ -137,6 +138,12 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
}
@Override
+ public boolean isAligned(Interval interval)
+ {
+ return bucket(interval.getStart()).equals(interval);
+ }
+
+ @Override
public byte[] getCacheKey()
{
return StringUtils.toUtf8(getPeriod() + ":" + getTimeZone() + ":" + getOrigin());
diff --git a/core/src/main/java/org/apache/druid/timeline/DataSegment.java b/core/src/main/java/org/apache/druid/timeline/DataSegment.java
index fb6a287..c427f6a 100644
--- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java
+++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java
@@ -56,7 +56,7 @@ import java.util.stream.Collectors;
* {@link SegmentId} of the segment.
*/
@PublicApi
-public class DataSegment implements Comparable<DataSegment>
+public class DataSegment implements Comparable<DataSegment>, Overshadowable<DataSegment>
{
/*
* The difference between this class and org.apache.druid.segment.Segment is that this class contains the segment
@@ -94,6 +94,29 @@ public class DataSegment implements Comparable<DataSegment>
private final long size;
public DataSegment(
+ SegmentId segmentId,
+ Map<String, Object> loadSpec,
+ List<String> dimensions,
+ List<String> metrics,
+ ShardSpec shardSpec,
+ Integer binaryVersion,
+ long size
+ )
+ {
+ this(
+ segmentId.getDataSource(),
+ segmentId.getInterval(),
+ segmentId.getVersion(),
+ loadSpec,
+ dimensions,
+ metrics,
+ shardSpec,
+ binaryVersion,
+ size
+ );
+ }
+
+ public DataSegment(
String dataSource,
Interval interval,
String version,
@@ -206,7 +229,8 @@ public class DataSegment implements Comparable<DataSegment>
return loadSpec;
}
- @JsonProperty
+ @JsonProperty("version")
+ @Override
public String getVersion()
{
return id.getVersion();
@@ -251,6 +275,50 @@ public class DataSegment implements Comparable<DataSegment>
return id;
}
+ @Override
+ public boolean overshadows(DataSegment other)
+ {
+ if (id.getDataSource().equals(other.id.getDataSource()) && id.getInterval().overlaps(other.id.getInterval())) {
+ final int majorVersionCompare = id.getVersion().compareTo(other.id.getVersion());
+ if (majorVersionCompare > 0) {
+ return true;
+ } else if (majorVersionCompare == 0) {
+ return includeRootPartitions(other) && getMinorVersion() > other.getMinorVersion();
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public int getStartRootPartitionId()
+ {
+ return shardSpec.getStartRootPartitionId();
+ }
+
+ @Override
+ public int getEndRootPartitionId()
+ {
+ return shardSpec.getEndRootPartitionId();
+ }
+
+ @Override
+ public short getMinorVersion()
+ {
+ return shardSpec.getMinorVersion();
+ }
+
+ @Override
+ public short getAtomicUpdateGroupSize()
+ {
+ return shardSpec.getAtomicUpdateGroupSize();
+ }
+
+ private boolean includeRootPartitions(DataSegment other)
+ {
+ return shardSpec.getStartRootPartitionId() <= other.shardSpec.getStartRootPartitionId()
+ && shardSpec.getEndRootPartitionId() >= other.shardSpec.getEndRootPartitionId();
+ }
+
public SegmentDescriptor toDescriptor()
{
return id.toDescriptor();
@@ -271,6 +339,11 @@ public class DataSegment implements Comparable<DataSegment>
return builder(this).metrics(metrics).build();
}
+ public DataSegment withShardSpec(ShardSpec newSpec)
+ {
+ return builder(this).shardSpec(newSpec).build();
+ }
+
public DataSegment withSize(long size)
{
return builder(this).size(size).build();
@@ -311,15 +384,13 @@ public class DataSegment implements Comparable<DataSegment>
public String toString()
{
return "DataSegment{" +
- "size=" + size +
- ", shardSpec=" + shardSpec +
- ", metrics=" + metrics +
- ", dimensions=" + dimensions +
- ", version='" + getVersion() + '\'' +
+ "binaryVersion=" + binaryVersion +
+ ", id=" + id +
", loadSpec=" + loadSpec +
- ", interval=" + getInterval() +
- ", dataSource='" + getDataSource() + '\'' +
- ", binaryVersion='" + binaryVersion + '\'' +
+ ", dimensions=" + dimensions +
+ ", metrics=" + metrics +
+ ", shardSpec=" + shardSpec +
+ ", size=" + size +
'}';
}
diff --git a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java
new file mode 100644
index 0000000..69b4336
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java
@@ -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.timeline;
+
+/**
+ * Interface to represent a class which can have overshadow relation between its instances.
+ * In {@link VersionedIntervalTimeline}, Overshadowable is used to represent each {@link DataSegment}
+ * which has the same major version in the same time chunk.
+ *
+ * An Overshadowable overshadows another if its root partition range contains that of another
+ * and has a higher minorVersion. For more details, check https://github.com/apache/incubator-druid/issues/7491.
+ */
+public interface Overshadowable<T extends Overshadowable>
+{
+ /**
+ * Returns true if this overshadowable overshadows the other.
+ */
+ default boolean overshadows(T other)
+ {
+ final int majorVersionCompare = getVersion().compareTo(other.getVersion());
+ if (majorVersionCompare == 0) {
+ return containsRootPartition(other) && getMinorVersion() > other.getMinorVersion();
+ } else {
+ return majorVersionCompare > 0;
+ }
+ }
+
+ default boolean containsRootPartition(T other)
+ {
+ return getStartRootPartitionId() <= other.getStartRootPartitionId()
+ && getEndRootPartitionId() >= other.getEndRootPartitionId();
+ }
+
+ /**
+ * All overshadowables have root partition range.
+ * First-generation overshadowables have (partitionId, partitionId + 1) as their root partition range.
+ * Non-first-generation overshadowables are the overshadowables that overwrite first or non-first generation
+ * overshadowables, and they have the merged root partition range of all overwritten first-generation overshadowables.
+ *
+ * Note that first-generation overshadowables can be overwritten by a single non-first-generation overshadowable
+ * if they have consecutive partitionId. Non-first-generation overshadowables can be overwritten by another
+ * if their root partition ranges are consecutive.
+ */
+ int getStartRootPartitionId();
+
+ /**
+ * See doc of {@link #getStartRootPartitionId()}.
+ */
+ int getEndRootPartitionId();
+
+ String getVersion();
+
+ short getMinorVersion();
+
+ /**
+ * Return the size of atomicUpdateGroup.
+ * An atomicUpdateGroup is a set of segments which should be updated all together atomically in
+ * {@link VersionedIntervalTimeline}.
+ */
+ short getAtomicUpdateGroupSize();
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentId.java b/core/src/main/java/org/apache/druid/timeline/SegmentId.java
index 8ac01e4..37fd68d 100644
--- a/core/src/main/java/org/apache/druid/timeline/SegmentId.java
+++ b/core/src/main/java/org/apache/druid/timeline/SegmentId.java
@@ -323,7 +323,6 @@ public final class SegmentId implements Comparable<SegmentId>
return new Interval(intervalStartMillis, intervalEndMillis, intervalChronology);
}
- @Nullable
public String getVersion()
{
return version;
diff --git a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java
index 6bdab5c..fb9e577 100644
--- a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java
+++ b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java
@@ -26,7 +26,7 @@ import javax.annotation.Nullable;
import java.util.List;
-public interface TimelineLookup<VersionType, ObjectType>
+public interface TimelineLookup<VersionType, ObjectType extends Overshadowable<ObjectType>>
{
/**
diff --git a/core/src/main/java/org/apache/druid/timeline/TimelineObjectHolder.java b/core/src/main/java/org/apache/druid/timeline/TimelineObjectHolder.java
index 3feca88..4aa1738 100644
--- a/core/src/main/java/org/apache/druid/timeline/TimelineObjectHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/TimelineObjectHolder.java
@@ -23,9 +23,11 @@ import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.timeline.partition.PartitionHolder;
import org.joda.time.Interval;
+import java.util.Objects;
+
/**
*/
-public class TimelineObjectHolder<VersionType, ObjectType> implements LogicalSegment
+public class TimelineObjectHolder<VersionType, ObjectType extends Overshadowable<ObjectType>> implements LogicalSegment
{
private final Interval interval;
private final Interval trueInterval;
@@ -74,6 +76,27 @@ public class TimelineObjectHolder<VersionType, ObjectType> implements LogicalSeg
}
@Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ TimelineObjectHolder<?, ?> that = (TimelineObjectHolder<?, ?>) o;
+ return Objects.equals(interval, that.interval) &&
+ Objects.equals(version, that.version) &&
+ Objects.equals(object, that.object);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(interval, version, object);
+ }
+
+ @Override
public String toString()
{
return "TimelineObjectHolder{" +
diff --git a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
index 8deed36..4998c41 100644
--- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
+++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
@@ -42,6 +42,7 @@ import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.NavigableMap;
import java.util.Objects;
import java.util.Set;
@@ -63,21 +64,27 @@ import java.util.stream.StreamSupport;
* a certain time period and when you do a lookup(), you are asking for all of the objects that you need to look
* at in order to get a correct answer about that time period.
*
- * The findOvershadowed() method returns a list of objects that will never be returned by a call to lookup() because
+ * The findFullyOvershadowed() method returns a list of objects that will never be returned by a call to lookup() because
* they are overshadowed by some other object. This can be used in conjunction with the add() and remove() methods
* to achieve "atomic" updates. First add new items, then check if those items caused anything to be overshadowed, if
* so, remove the overshadowed elements and you have effectively updated your data set without any user impact.
*/
-public class VersionedIntervalTimeline<VersionType, ObjectType> implements TimelineLookup<VersionType, ObjectType>
+public class VersionedIntervalTimeline<VersionType, ObjectType extends Overshadowable<ObjectType>> implements TimelineLookup<VersionType, ObjectType>
{
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
- final NavigableMap<Interval, TimelineEntry> completePartitionsTimeline = new TreeMap<Interval, TimelineEntry>(
+ // Below timelines stores only *visible* timelineEntries
+ // adjusted interval -> timelineEntry
+ private final NavigableMap<Interval, TimelineEntry> completePartitionsTimeline = new TreeMap<>(
Comparators.intervalsByStartThenEnd()
);
- final NavigableMap<Interval, TimelineEntry> incompletePartitionsTimeline = new TreeMap<Interval, TimelineEntry>(
+ // IncompletePartitionsTimeline also includes completePartitionsTimeline
+ // adjusted interval -> timelineEntry
+ @VisibleForTesting
+ final NavigableMap<Interval, TimelineEntry> incompletePartitionsTimeline = new TreeMap<>(
Comparators.intervalsByStartThenEnd()
);
+ // true interval -> version -> timelineEntry
private final Map<Interval, TreeMap<VersionType, TimelineEntry>> allTimelineEntries = new HashMap<>();
private final AtomicInteger numObjects = new AtomicInteger();
@@ -113,14 +120,13 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
);
}
- @VisibleForTesting
public Map<Interval, TreeMap<VersionType, TimelineEntry>> getAllTimelineEntries()
{
return allTimelineEntries;
}
/**
- * Returns a lazy collection with all objects (including overshadowed, see {@link #findOvershadowed}) in this
+ * Returns a lazy collection with all objects (including overshadowed, see {@link #findFullyOvershadowed}) in this
* VersionedIntervalTimeline to be used for iteration or {@link Collection#stream()} transformation. The order of
* objects in this collection is unspecified.
*
@@ -189,7 +195,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
// "isComplete" is O(objects in holder) so defer it to the end of addAll.
- for (Map.Entry<TimelineEntry, Interval> entry : allEntries.entrySet()) {
+ for (Entry<TimelineEntry, Interval> entry : allEntries.entrySet()) {
Interval interval = entry.getValue();
if (entry.getKey().getPartitionHolder().isComplete()) {
@@ -207,9 +213,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
@Nullable
public PartitionChunk<ObjectType> remove(Interval interval, VersionType version, PartitionChunk<ObjectType> chunk)
{
+ lock.writeLock().lock();
try {
- lock.writeLock().lock();
-
Map<VersionType, TimelineEntry> versionEntries = allTimelineEntries.get(interval);
if (versionEntries == null) {
return null;
@@ -246,9 +251,9 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
@Override
public @Nullable PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version)
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
- for (Map.Entry<Interval, TreeMap<VersionType, TimelineEntry>> entry : allTimelineEntries.entrySet()) {
+ for (Entry<Interval, TreeMap<VersionType, TimelineEntry>> entry : allTimelineEntries.entrySet()) {
if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) {
TimelineEntry foundEntry = entry.getValue().get(version);
if (foundEntry != null) {
@@ -276,8 +281,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
@Override
public List<TimelineObjectHolder<VersionType, ObjectType>> lookup(Interval interval)
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
return lookup(interval, false);
}
finally {
@@ -288,8 +293,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
@Override
public List<TimelineObjectHolder<VersionType, ObjectType>> lookupWithIncompletePartitions(Interval interval)
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
return lookup(interval, true);
}
finally {
@@ -299,8 +304,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
public boolean isEmpty()
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
return completePartitionsTimeline.isEmpty();
}
finally {
@@ -310,8 +315,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
public TimelineObjectHolder<VersionType, ObjectType> first()
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
return timelineEntryToObjectHolder(completePartitionsTimeline.firstEntry().getValue());
}
finally {
@@ -321,8 +326,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
public TimelineObjectHolder<VersionType, ObjectType> last()
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
return timelineEntryToObjectHolder(completePartitionsTimeline.lastEntry().getValue());
}
finally {
@@ -344,20 +349,19 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
* This method should be deduplicated with DataSourcesSnapshot.determineOvershadowedSegments(): see
* https://github.com/apache/incubator-druid/issues/8070.
*/
- public Set<TimelineObjectHolder<VersionType, ObjectType>> findOvershadowed()
+ public Set<TimelineObjectHolder<VersionType, ObjectType>> findFullyOvershadowed()
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
- Set<TimelineObjectHolder<VersionType, ObjectType>> retVal = new HashSet<>();
-
- Map<Interval, Map<VersionType, TimelineEntry>> overShadowed = new HashMap<>();
+ // 1. Put all timelineEntries and remove all visible entries to find out only non-visible timelineEntries.
+ final Map<Interval, Map<VersionType, TimelineEntry>> overShadowed = new HashMap<>();
for (Map.Entry<Interval, TreeMap<VersionType, TimelineEntry>> versionEntry : allTimelineEntries.entrySet()) {
@SuppressWarnings("unchecked")
Map<VersionType, TimelineEntry> versionCopy = (TreeMap) versionEntry.getValue().clone();
overShadowed.put(versionEntry.getKey(), versionCopy);
}
- for (Map.Entry<Interval, TimelineEntry> entry : completePartitionsTimeline.entrySet()) {
+ for (Entry<Interval, TimelineEntry> entry : completePartitionsTimeline.entrySet()) {
Map<VersionType, TimelineEntry> versionEntry = overShadowed.get(entry.getValue().getTrueInterval());
if (versionEntry != null) {
versionEntry.remove(entry.getValue().getVersion());
@@ -367,7 +371,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
}
- for (Map.Entry<Interval, TimelineEntry> entry : incompletePartitionsTimeline.entrySet()) {
+ for (Entry<Interval, TimelineEntry> entry : incompletePartitionsTimeline.entrySet()) {
Map<VersionType, TimelineEntry> versionEntry = overShadowed.get(entry.getValue().getTrueInterval());
if (versionEntry != null) {
versionEntry.remove(entry.getValue().getVersion());
@@ -377,10 +381,25 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
}
- for (Map.Entry<Interval, Map<VersionType, TimelineEntry>> versionEntry : overShadowed.entrySet()) {
- for (Map.Entry<VersionType, TimelineEntry> entry : versionEntry.getValue().entrySet()) {
- TimelineEntry object = entry.getValue();
- retVal.add(timelineEntryToObjectHolder(object));
+ final Set<TimelineObjectHolder<VersionType, ObjectType>> retVal = new HashSet<>();
+ for (Entry<Interval, Map<VersionType, TimelineEntry>> versionEntry : overShadowed.entrySet()) {
+ for (Entry<VersionType, TimelineEntry> entry : versionEntry.getValue().entrySet()) {
+ final TimelineEntry timelineEntry = entry.getValue();
+ retVal.add(timelineEntryToObjectHolder(timelineEntry));
+ }
+ }
+
+ // 2. Visible timelineEntries can also have overshadowed segments. Add them to the result too.
+ for (TimelineEntry entry : incompletePartitionsTimeline.values()) {
+ final List<PartitionChunk<ObjectType>> entryOvershadowed = entry.partitionHolder.getOvershadowed();
+ if (!entryOvershadowed.isEmpty()) {
+ retVal.add(
+ new TimelineObjectHolder<>(
+ entry.trueInterval,
+ entry.version,
+ new PartitionHolder<>(entryOvershadowed)
+ )
+ );
}
}
@@ -391,14 +410,23 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
}
- public boolean isOvershadowed(Interval interval, VersionType version)
+ public boolean isOvershadowed(Interval interval, VersionType version, ObjectType object)
{
+ lock.readLock().lock();
try {
- lock.readLock().lock();
-
TimelineEntry entry = completePartitionsTimeline.get(interval);
if (entry != null) {
- return versionComparator.compare(version, entry.getVersion()) < 0;
+ final int majorVersionCompare = versionComparator.compare(version, entry.getVersion());
+ if (majorVersionCompare == 0) {
+ for (PartitionChunk<ObjectType> chunk : entry.partitionHolder) {
+ if (chunk.getObject().overshadows(object)) {
+ return true;
+ }
+ }
+ return false;
+ } else {
+ return majorVersionCompare < 0;
+ }
}
Interval lower = completePartitionsTimeline.floorKey(
@@ -414,13 +442,23 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
do {
if (curr == null || //no further keys
- (prev != null && curr.getStartMillis() > prev.getEndMillis()) || //a discontinuity
- //lower or same version
- versionComparator.compare(version, completePartitionsTimeline.get(curr).getVersion()) >= 0
- ) {
+ (prev != null && curr.getStartMillis() > prev.getEndMillis()) //a discontinuity
+ ) {
return false;
}
+ final TimelineEntry timelineEntry = completePartitionsTimeline.get(curr);
+ final int versionCompare = versionComparator.compare(version, timelineEntry.getVersion());
+
+ //lower or same version
+ if (versionCompare > 0) {
+ return false;
+ } else if (versionCompare == 0) {
+ if (timelineEntry.partitionHolder.stream().noneMatch(chunk -> chunk.getObject().overshadows(object))) {
+ return false;
+ }
+ }
+
prev = curr;
curr = completePartitionsTimeline.higherKey(curr);
@@ -490,39 +528,64 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
while (entryInterval != null && currKey != null && currKey.overlaps(entryInterval)) {
- Interval nextKey = timeline.higherKey(currKey);
+ final Interval nextKey = timeline.higherKey(currKey);
- int versionCompare = versionComparator.compare(
+ final int versionCompare = versionComparator.compare(
entry.getVersion(),
timeline.get(currKey).getVersion()
);
if (versionCompare < 0) {
+ // since the entry version is lower than the existing one, the existing one overwrites the given entry
+ // if overlapped.
if (currKey.contains(entryInterval)) {
+ // the version of the entry of currKey is larger than that of the given entry. Discard it
return true;
} else if (currKey.getStart().isBefore(entryInterval.getStart())) {
+ // | entry |
+ // | cur |
+ // => |new|
entryInterval = new Interval(currKey.getEnd(), entryInterval.getEnd());
} else {
+ // | entry |
+ // | cur |
+ // => |new|
addIntervalToTimeline(new Interval(entryInterval.getStart(), currKey.getStart()), entry, timeline);
+ // | entry |
+ // | cur |
+ // => |new|
if (entryInterval.getEnd().isAfter(currKey.getEnd())) {
entryInterval = new Interval(currKey.getEnd(), entryInterval.getEnd());
} else {
- entryInterval = null; // discard this entry
+ // Discard this entry since there is no portion of the entry interval that goes past the end of the curr
+ // key interval.
+ entryInterval = null;
}
}
} else if (versionCompare > 0) {
- TimelineEntry oldEntry = timeline.remove(currKey);
+ // since the entry version is greater than the existing one, the given entry overwrites the existing one
+ // if overlapped.
+ final TimelineEntry oldEntry = timeline.remove(currKey);
if (currKey.contains(entryInterval)) {
+ // | cur |
+ // | entry |
+ // => |old| new |old|
addIntervalToTimeline(new Interval(currKey.getStart(), entryInterval.getStart()), oldEntry, timeline);
addIntervalToTimeline(new Interval(entryInterval.getEnd(), currKey.getEnd()), oldEntry, timeline);
addIntervalToTimeline(entryInterval, entry, timeline);
return true;
} else if (currKey.getStart().isBefore(entryInterval.getStart())) {
+ // | cur |
+ // | entry |
+ // => |old|
addIntervalToTimeline(new Interval(currKey.getStart(), entryInterval.getStart()), oldEntry, timeline);
} else if (entryInterval.getEnd().isBefore(currKey.getEnd())) {
+ // | cur |
+ // | entry |
+ // => |old|
addIntervalToTimeline(new Interval(entryInterval.getEnd(), currKey.getEnd()), oldEntry, timeline);
}
} else {
@@ -570,9 +633,9 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
TimelineEntry removed = timeline.get(interval);
if (removed == null) {
- Iterator<Map.Entry<Interval, TimelineEntry>> iter = timeline.entrySet().iterator();
+ Iterator<Entry<Interval, TimelineEntry>> iter = timeline.entrySet().iterator();
while (iter.hasNext()) {
- Map.Entry<Interval, TimelineEntry> timelineEntry = iter.next();
+ Entry<Interval, TimelineEntry> timelineEntry = iter.next();
if (timelineEntry.getValue() == entry) {
intervalsToRemove.add(timelineEntry.getKey());
}
@@ -594,7 +657,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
{
timeline.remove(interval);
- for (Map.Entry<Interval, TreeMap<VersionType, TimelineEntry>> versionEntry : allTimelineEntries.entrySet()) {
+ for (Entry<Interval, TreeMap<VersionType, TimelineEntry>> versionEntry : allTimelineEntries.entrySet()) {
if (versionEntry.getKey().overlap(interval) != null) {
if (incompleteOk) {
add(timeline, versionEntry.getKey(), versionEntry.getValue().lastEntry().getValue());
@@ -613,12 +676,12 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
private List<TimelineObjectHolder<VersionType, ObjectType>> lookup(Interval interval, boolean incompleteOk)
{
- List<TimelineObjectHolder<VersionType, ObjectType>> retVal = new ArrayList<TimelineObjectHolder<VersionType, ObjectType>>();
+ List<TimelineObjectHolder<VersionType, ObjectType>> retVal = new ArrayList<>();
NavigableMap<Interval, TimelineEntry> timeline = (incompleteOk)
? incompletePartitionsTimeline
: completePartitionsTimeline;
- for (Map.Entry<Interval, TimelineEntry> entry : timeline.entrySet()) {
+ for (Entry<Interval, TimelineEntry> entry : timeline.entrySet()) {
Interval timelineInterval = entry.getKey();
TimelineEntry val = entry.getValue();
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java
new file mode 100644
index 0000000..aed7f15
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java
@@ -0,0 +1,175 @@
+/*
+ * 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.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.timeline.Overshadowable;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * A set of {@link PartitionChunk}s which should be atomically visible or not in the timeline.
+ * This is usually a set of single partitionChunk for first-generation segments.
+ * For non-first-generation segments generated by overwriting tasks, segments of the same interval generated by
+ * the same task become an atomicUpdateGroup. As a result, all segments in an atomicUpdateGroup have the same
+ * rootPartitionp range, majorVersion, minorVersion, and atomicUpdateGroupSize.
+ */
+class AtomicUpdateGroup<T extends Overshadowable<T>> implements Overshadowable<AtomicUpdateGroup<T>>
+{
+ // Perhaps it would be worth to store these in a map of (partitionId -> partitionChunk)
+ // because sometimes we need to search for a particular partitionChunk corresponding to a partitionId.
+ // However, there's a tradeoff between time and space. Storing in a map would be faster than storing in a list,
+ // but it would take at least additional 4 bytes per chunk to store its key.
+ // This may matter if there are a lot of segments to keep in memory as in brokers or the coordinator.
+ private final List<PartitionChunk<T>> chunks = new ArrayList<>();
+
+ public AtomicUpdateGroup(PartitionChunk<T> chunk)
+ {
+ this.chunks.add(chunk);
+ }
+
+ public void add(PartitionChunk<T> chunk)
+ {
+ if (isFull()) {
+ throw new IAE("Can't add more chunk[%s] to atomicUpdateGroup[%s]", chunk, chunks);
+ }
+ if (!isEmpty() && !isSameAtomicUpdateGroup(chunks.get(0), chunk)) {
+ throw new IAE("Can't add chunk[%s] to a different atomicUpdateGroup[%s]", chunk, chunks);
+ }
+ for (PartitionChunk<T> existing : chunks) {
+ if (existing.equals(chunk)) {
+ return;
+ }
+ }
+ chunks.add(chunk);
+ }
+
+ public void remove(PartitionChunk<T> chunk)
+ {
+ if (chunks.isEmpty()) {
+ throw new ISE("Can't remove chunk[%s] from empty atomicUpdateGroup", chunk);
+ }
+
+ if (!isSameAtomicUpdateGroup(chunks.get(0), chunk)) {
+ throw new IAE("Can't remove chunk[%s] from a different atomicUpdateGroup[%s]", chunk, chunks);
+ }
+
+ chunks.remove(chunk);
+ }
+
+ public boolean isFull()
+ {
+ return !isEmpty() && chunks.size() == chunks.get(0).getObject().getAtomicUpdateGroupSize();
+ }
+
+ public boolean isEmpty()
+ {
+ return chunks.isEmpty();
+ }
+
+ public List<PartitionChunk<T>> getChunks()
+ {
+ return chunks;
+ }
+
+ @Nullable
+ public PartitionChunk<T> findChunk(int partitionId)
+ {
+ return chunks.stream().filter(chunk -> chunk.getChunkNumber() == partitionId).findFirst().orElse(null);
+ }
+
+ @Override
+ public int getStartRootPartitionId()
+ {
+ Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup");
+ return chunks.get(0).getObject().getStartRootPartitionId();
+ }
+
+ @Override
+ public int getEndRootPartitionId()
+ {
+ Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup");
+ return chunks.get(0).getObject().getEndRootPartitionId();
+ }
+
+ @Override
+ public String getVersion()
+ {
+ Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup");
+ return chunks.get(0).getObject().getVersion();
+ }
+
+ @Override
+ public short getMinorVersion()
+ {
+ Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup");
+ return chunks.get(0).getObject().getMinorVersion();
+ }
+
+ @Override
+ public short getAtomicUpdateGroupSize()
+ {
+ Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup");
+ return chunks.get(0).getObject().getAtomicUpdateGroupSize();
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ AtomicUpdateGroup<?> that = (AtomicUpdateGroup<?>) o;
+ return Objects.equals(chunks, that.chunks);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(chunks);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "AtomicUpdateGroup{" +
+ "chunks=" + chunks +
+ '}';
+ }
+
+ private static <T extends Overshadowable<T>> boolean isSameAtomicUpdateGroup(
+ PartitionChunk<T> c1,
+ PartitionChunk<T> c2
+ )
+ {
+ return c1.getObject().getStartRootPartitionId() == c2.getObject().getStartRootPartitionId()
+ && c1.getObject().getEndRootPartitionId() == c2.getObject().getEndRootPartitionId()
+ && c1.getObject().getMinorVersion() == c2.getObject().getMinorVersion()
+ && c1.getObject().getAtomicUpdateGroupSize() == c2.getObject().getAtomicUpdateGroupSize();
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java
index 0d85423..88dfc83 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java
@@ -47,8 +47,8 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec
@JsonCreator
public HashBasedNumberedShardSpec(
- @JsonProperty("partitionNum") int partitionNum,
- @JsonProperty("partitions") int partitions,
+ @JsonProperty("partitionNum") int partitionNum, // partitionId
+ @JsonProperty("partitions") int partitions, // # of partitions
@JsonProperty("partitionDimensions") @Nullable List<String> partitionDimensions,
@JacksonInject ObjectMapper jsonMapper
)
@@ -65,6 +65,12 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec
}
@Override
+ public boolean isCompatible(Class<? extends ShardSpec> other)
+ {
+ return other == HashBasedNumberedShardSpec.class;
+ }
+
+ @Override
public boolean isInChunk(long timestamp, InputRow inputRow)
{
return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % getPartitions() == 0;
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecFactory.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecFactory.java
new file mode 100644
index 0000000..eaaa2c7
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecFactory.java
@@ -0,0 +1,101 @@
+/*
+ * 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.List;
+import java.util.Objects;
+
+public class HashBasedNumberedShardSpecFactory implements ShardSpecFactory
+{
+ @Nullable
+ private final List<String> partitionDimensions;
+ private final int numPartitions;
+
+ @JsonCreator
+ public HashBasedNumberedShardSpecFactory(
+ @JsonProperty("partitionDimensions") @Nullable List<String> partitionDimensions,
+ @JsonProperty("numPartitions") int numPartitions
+ )
+ {
+ this.partitionDimensions = partitionDimensions;
+ this.numPartitions = numPartitions;
+ }
+
+ @Nullable
+ @JsonProperty
+ public List<String> getPartitionDimensions()
+ {
+ return partitionDimensions;
+ }
+
+ @JsonProperty public int getNumPartitions()
+ {
+ return numPartitions;
+ }
+
+ @Override
+ public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
+ {
+ final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId;
+ return new HashBasedNumberedShardSpec(
+ prevSpec == null ? 0 : prevSpec.getPartitionNum() + 1,
+ numPartitions,
+ partitionDimensions,
+ objectMapper
+ );
+ }
+
+ @Override
+ public ShardSpec create(ObjectMapper objectMapper, int partitionId)
+ {
+ return new HashBasedNumberedShardSpec(partitionId, numPartitions, partitionDimensions, objectMapper);
+ }
+
+ @Override
+ public Class<? extends ShardSpec> getShardSpecClass()
+ {
+ return HashBasedNumberedShardSpec.class;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ HashBasedNumberedShardSpecFactory that = (HashBasedNumberedShardSpecFactory) o;
+ return numPartitions == that.numPartitions &&
+ Objects.equals(partitionDimensions, that.partitionDimensions);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(partitionDimensions, numPartitions);
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
index 5003f65..65c1b0f 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
@@ -19,9 +19,11 @@
package org.apache.druid.timeline.partition;
+import org.apache.druid.timeline.Overshadowable;
+
/**
*/
-public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
+public class ImmutablePartitionHolder<T extends Overshadowable<T>> extends PartitionHolder<T>
{
public ImmutablePartitionHolder(PartitionHolder<T> partitionHolder)
{
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/IntegerPartitionChunk.java b/core/src/main/java/org/apache/druid/timeline/partition/IntegerPartitionChunk.java
index 986f8fe..8855f01 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/IntegerPartitionChunk.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/IntegerPartitionChunk.java
@@ -19,16 +19,18 @@
package org.apache.druid.timeline.partition;
+import org.apache.druid.timeline.Overshadowable;
+
/**
*/
-public class IntegerPartitionChunk<T> implements PartitionChunk<T>
+public class IntegerPartitionChunk<T extends Overshadowable> implements PartitionChunk<T>
{
private final Integer start;
private final Integer end;
private final int chunkNumber;
private final T object;
- public static <T> IntegerPartitionChunk<T> make(Integer start, Integer end, int chunkNumber, T obj)
+ public static <T extends Overshadowable> IntegerPartitionChunk<T> make(Integer start, Integer end, int chunkNumber, T obj)
{
return new IntegerPartitionChunk<T>(start, end, chunkNumber, obj);
}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java
index d0e60c2..1ebb24e 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java
@@ -68,6 +68,12 @@ public final class LinearShardSpec implements ShardSpec
}
@Override
+ public boolean isCompatible(Class<? extends ShardSpec> other)
+ {
+ return other == LinearShardSpec.class;
+ }
+
+ @Override
public <T> PartitionChunk<T> createChunk(T obj)
{
return new LinearPartitionChunk<>(partitionNum, obj);
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpecFactory.java
similarity index 52%
copy from core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
copy to core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpecFactory.java
index 5003f65..b6340ec 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpecFactory.java
@@ -19,24 +19,40 @@
package org.apache.druid.timeline.partition;
-/**
- */
-public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+
+public class LinearShardSpecFactory implements ShardSpecFactory
{
- public ImmutablePartitionHolder(PartitionHolder<T> partitionHolder)
+ private static final LinearShardSpecFactory INSTANCE = new LinearShardSpecFactory();
+
+ public static LinearShardSpecFactory instance()
+ {
+ return INSTANCE;
+ }
+
+ private LinearShardSpecFactory()
+ {
+ }
+
+ @Override
+ public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
{
- super(partitionHolder);
+ return new LinearShardSpec(
+ specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1
+ );
}
@Override
- public PartitionChunk<T> remove(PartitionChunk<T> tPartitionChunk)
+ public ShardSpec create(ObjectMapper objectMapper, int partitionId)
{
- throw new UnsupportedOperationException();
+ return new LinearShardSpec(partitionId);
}
@Override
- public boolean add(PartitionChunk<T> tPartitionChunk)
+ public Class<? extends ShardSpec> getShardSpecClass()
{
- throw new UnsupportedOperationException();
+ return LinearShardSpec.class;
}
}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java
index 7fb2aab..dde9216 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java
@@ -49,7 +49,7 @@ public class NoneShardSpec implements ShardSpec
@Override
public <T> PartitionChunk<T> createChunk(T obj)
{
- return new SingleElementPartitionChunk<T>(obj);
+ return new SingleElementPartitionChunk<>(obj);
}
@Override
@@ -84,6 +84,12 @@ public class NoneShardSpec implements ShardSpec
}
@Override
+ public boolean isCompatible(Class<? extends ShardSpec> other)
+ {
+ return other == NoneShardSpec.class;
+ }
+
+ @Override
public boolean equals(Object obj)
{
return obj instanceof NoneShardSpec;
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java
new file mode 100644
index 0000000..d5b3657
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java
@@ -0,0 +1,229 @@
+/*
+ * 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.google.common.base.Preconditions;
+import com.google.common.collect.RangeSet;
+import org.apache.druid.data.input.InputRow;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * ShardSpec for segments which overshadow others with their minorVersion.
+ */
+public class NumberedOverwriteShardSpec implements OverwriteShardSpec
+{
+ private final int partitionId;
+
+ private final short startRootPartitionId;
+ private final short endRootPartitionId; // exclusive
+ private final short minorVersion;
+ private final short atomicUpdateGroupSize; // number of segments in atomicUpdateGroup
+
+ @JsonCreator
+ public NumberedOverwriteShardSpec(
+ @JsonProperty("partitionId") int partitionId,
+ @JsonProperty("startRootPartitionId") int startRootPartitionId,
+ @JsonProperty("endRootPartitionId") int endRootPartitionId,
+ @JsonProperty("minorVersion") short minorVersion,
+ @JsonProperty("atomicUpdateGroupSize") short atomicUpdateGroupSize
+ )
+ {
+ Preconditions.checkArgument(
+ partitionId >= PartitionIds.NON_ROOT_GEN_START_PARTITION_ID
+ && partitionId < PartitionIds.NON_ROOT_GEN_END_PARTITION_ID,
+ "partitionNum[%s] >= %s && partitionNum[%s] < %s",
+ partitionId,
+ PartitionIds.NON_ROOT_GEN_START_PARTITION_ID,
+ partitionId,
+ PartitionIds.NON_ROOT_GEN_END_PARTITION_ID
+ );
+ Preconditions.checkArgument(
+ startRootPartitionId >= PartitionIds.ROOT_GEN_START_PARTITION_ID
+ && startRootPartitionId < PartitionIds.ROOT_GEN_END_PARTITION_ID,
+ "startRootPartitionId[%s] >= %s && startRootPartitionId[%s] < %s",
+ startRootPartitionId,
+ PartitionIds.ROOT_GEN_START_PARTITION_ID,
+ startRootPartitionId,
+ PartitionIds.ROOT_GEN_END_PARTITION_ID
+ );
+ Preconditions.checkArgument(
+ endRootPartitionId >= PartitionIds.ROOT_GEN_START_PARTITION_ID
+ && endRootPartitionId < PartitionIds.ROOT_GEN_END_PARTITION_ID,
+ "endRootPartitionId[%s] >= %s && endRootPartitionId[%s] < %s",
+ endRootPartitionId,
+ PartitionIds.ROOT_GEN_START_PARTITION_ID,
+ endRootPartitionId,
+ PartitionIds.ROOT_GEN_END_PARTITION_ID
+ );
+ Preconditions.checkArgument(minorVersion > 0, "minorVersion[%s] > 0", minorVersion);
+ Preconditions.checkArgument(
+ atomicUpdateGroupSize > 0 || atomicUpdateGroupSize == PartitionIds.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE,
+ "atomicUpdateGroupSize[%s] > 0 or == %s",
+ atomicUpdateGroupSize,
+ PartitionIds.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE
+ );
+
+ this.partitionId = partitionId;
+ this.startRootPartitionId = (short) startRootPartitionId;
+ this.endRootPartitionId = (short) endRootPartitionId;
+ this.minorVersion = minorVersion;
+ this.atomicUpdateGroupSize = atomicUpdateGroupSize;
+ }
+
+ public NumberedOverwriteShardSpec(
+ int partitionId,
+ int startRootPartitionId,
+ int endRootPartitionId,
+ short minorVersion
+ )
+ {
+ this(
+ partitionId,
+ startRootPartitionId,
+ endRootPartitionId,
+ minorVersion,
+ PartitionIds.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE
+ );
+ }
+
+ @Override
+ public OverwriteShardSpec withAtomicUpdateGroupSize(short atomicUpdateGroupSize)
+ {
+ return new NumberedOverwriteShardSpec(
+ this.partitionId,
+ this.startRootPartitionId,
+ this.endRootPartitionId,
+ this.minorVersion,
+ atomicUpdateGroupSize
+ );
+ }
+
+ @Override
+ public <T> PartitionChunk<T> createChunk(T obj)
+ {
+ return new NumberedOverwritingPartitionChunk<>(partitionId, obj);
+ }
+
+ @Override
+ public boolean isInChunk(long timestamp, InputRow inputRow)
+ {
+ return true;
+ }
+
+ @JsonProperty("partitionId")
+ @Override
+ public int getPartitionNum()
+ {
+ return partitionId;
+ }
+
+ @JsonProperty
+ @Override
+ public int getStartRootPartitionId()
+ {
+ return Short.toUnsignedInt(startRootPartitionId);
+ }
+
+ @JsonProperty
+ @Override
+ public int getEndRootPartitionId()
+ {
+ return Short.toUnsignedInt(endRootPartitionId);
+ }
+
+ @JsonProperty
+ @Override
+ public short getMinorVersion()
+ {
+ return minorVersion;
+ }
+
+ @JsonProperty
+ @Override
+ public short getAtomicUpdateGroupSize()
+ {
+ return atomicUpdateGroupSize;
+ }
+
+ @Override
+ public ShardSpecLookup getLookup(List<ShardSpec> shardSpecs)
+ {
+ return (long timestamp, InputRow row) -> shardSpecs.get(0);
+ }
+
+ @Override
+ public List<String> getDomainDimensions()
+ {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public boolean possibleInDomain(Map<String, RangeSet<String>> domain)
+ {
+ return true;
+ }
+
+ @Override
+ public boolean isCompatible(Class<? extends ShardSpec> other)
+ {
+ return other == NumberedOverwriteShardSpec.class || other == NumberedShardSpec.class;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ NumberedOverwriteShardSpec that = (NumberedOverwriteShardSpec) o;
+ return partitionId == that.partitionId &&
+ startRootPartitionId == that.startRootPartitionId &&
+ endRootPartitionId == that.endRootPartitionId &&
+ minorVersion == that.minorVersion &&
+ atomicUpdateGroupSize == that.atomicUpdateGroupSize;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(partitionId, startRootPartitionId, endRootPartitionId, minorVersion, atomicUpdateGroupSize);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "NumberedOverwriteShardSpec{" +
+ "partitionId=" + partitionId +
+ ", startRootPartitionId=" + startRootPartitionId +
+ ", endRootPartitionId=" + endRootPartitionId +
+ ", minorVersion=" + minorVersion +
+ ", atomicUpdateGroupSize=" + atomicUpdateGroupSize +
+ '}';
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java
new file mode 100644
index 0000000..3c6c982
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java
@@ -0,0 +1,118 @@
+/*
+ * 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.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.IAE;
+
+import java.util.Objects;
+
+/**
+ * PartitionChunk corresponding to {@link NumberedOverwriteShardSpec}
+ */
+public class NumberedOverwritingPartitionChunk<T> implements PartitionChunk<T>
+{
+ private final int chunkId;
+ private final T object;
+
+ public NumberedOverwritingPartitionChunk(int chunkId, T object)
+ {
+ Preconditions.checkArgument(
+ chunkId >= PartitionIds.NON_ROOT_GEN_START_PARTITION_ID && chunkId < PartitionIds.NON_ROOT_GEN_END_PARTITION_ID,
+ "partitionNum[%s] >= %s && partitionNum[%s] < %s",
+ chunkId,
+ PartitionIds.NON_ROOT_GEN_START_PARTITION_ID,
+ chunkId,
+ PartitionIds.NON_ROOT_GEN_END_PARTITION_ID
+ );
+
+ this.chunkId = chunkId;
+ this.object = object;
+ }
+
+ @Override
+ public T getObject()
+ {
+ return object;
+ }
+
+ @Override
+ public boolean abuts(PartitionChunk<T> other)
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean isStart()
+ {
+ return true;
+ }
+
+ @Override
+ public boolean isEnd()
+ {
+ return true;
+ }
+
+ @Override
+ public int getChunkNumber()
+ {
+ return chunkId;
+ }
+
+ @Override
+ public int compareTo(PartitionChunk<T> o)
+ {
+ if (o instanceof NumberedOverwritingPartitionChunk) {
+ final NumberedOverwritingPartitionChunk<T> that = (NumberedOverwritingPartitionChunk<T>) o;
+ return Integer.compare(chunkId, that.chunkId);
+ } else {
+ throw new IAE("Cannot compare against [%s]", o.getClass().getName());
+ }
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ NumberedOverwritingPartitionChunk<?> that = (NumberedOverwritingPartitionChunk<?>) o;
+ return chunkId == that.chunkId;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(chunkId);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "NumberedOverwritingPartitionChunk{" +
+ "chunkId=" + chunkId +
+ ", object=" + object +
+ '}';
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java
new file mode 100644
index 0000000..764917c
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.timeline.partition;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+
+public class NumberedOverwritingShardSpecFactory implements ShardSpecFactory
+{
+ private final int startRootPartitionId;
+ private final int endRootPartitionId;
+ private final short minorVersion;
+
+ @JsonCreator
+ public NumberedOverwritingShardSpecFactory(
+ @JsonProperty("startRootPartitionId") int startRootPartitionId,
+ @JsonProperty("endRootPartitionId") int endRootPartitionId,
+ @JsonProperty("minorVersion") short minorVersion
+ )
+ {
+ this.startRootPartitionId = startRootPartitionId;
+ this.endRootPartitionId = endRootPartitionId;
+ this.minorVersion = minorVersion;
+ }
+
+ @JsonProperty
+ public int getStartRootPartitionId()
+ {
+ return startRootPartitionId;
+ }
+
+ @JsonProperty
+ public int getEndRootPartitionId()
+ {
+ return endRootPartitionId;
+ }
+
+ @JsonProperty
+ public short getMinorVersion()
+ {
+ return minorVersion;
+ }
+
+ @Override
+ public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
+ {
+ // specOfPreviousMaxPartitionId is the max partitionId of the same shardSpec
+ // and could be null if all existing segments are first-generation segments.
+ return new NumberedOverwriteShardSpec(
+ specOfPreviousMaxPartitionId == null
+ ? PartitionIds.NON_ROOT_GEN_START_PARTITION_ID
+ : specOfPreviousMaxPartitionId.getPartitionNum() + 1,
+ startRootPartitionId,
+ endRootPartitionId,
+ minorVersion
+ );
+ }
+
+ @Override
+ public ShardSpec create(ObjectMapper objectMapper, int partitionId)
+ {
+ return new NumberedOverwriteShardSpec(partitionId, startRootPartitionId, endRootPartitionId, minorVersion);
+ }
+
+ @Override
+ public Class<? extends ShardSpec> getShardSpecClass()
+ {
+ return NumberedOverwriteShardSpec.class;
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartitionChunk.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartitionChunk.java
index 5ab89f6..2d32532 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartitionChunk.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartitionChunk.java
@@ -123,4 +123,14 @@ public class NumberedPartitionChunk<T> implements PartitionChunk<T>
{
return Objects.hashCode(chunks, chunkNumber);
}
+
+ @Override
+ public String toString()
+ {
+ return "NumberedPartitionChunk{" +
+ "chunkNumber=" + chunkNumber +
+ ", chunks=" + chunks +
+ ", object=" + object +
+ '}';
+ }
}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java
index 8b555e2..d6f98d7 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java
@@ -83,6 +83,12 @@ public class NumberedShardSpec implements ShardSpec
return true;
}
+ @Override
+ public boolean isCompatible(Class<? extends ShardSpec> other)
+ {
+ return other == NumberedShardSpec.class || other == NumberedOverwriteShardSpec.class;
+ }
+
@JsonProperty("partitions")
public int getPartitions()
{
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpecFactory.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpecFactory.java
new file mode 100644
index 0000000..486f9ed
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpecFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.timeline.partition;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+
+public class NumberedShardSpecFactory implements ShardSpecFactory
+{
+ private static final NumberedShardSpecFactory INSTANCE = new NumberedShardSpecFactory();
+
+ public static NumberedShardSpecFactory instance()
+ {
+ return INSTANCE;
+ }
+
+ private NumberedShardSpecFactory()
+ {
+ }
+
+ @Override
+ public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId)
+ {
+ if (specOfPreviousMaxPartitionId == null) {
+ return new NumberedShardSpec(0, 0);
+ } else {
+ final NumberedShardSpec prevSpec = (NumberedShardSpec) specOfPreviousMaxPartitionId;
+ return new NumberedShardSpec(prevSpec.getPartitionNum() + 1, prevSpec.getPartitions());
+ }
+ }
+
+ @Override
+ public ShardSpec create(ObjectMapper objectMapper, int partitionId)
+ {
+ return new NumberedShardSpec(partitionId, 0);
+ }
+
+ @Override
+ public Class<? extends ShardSpec> getShardSpecClass()
+ {
+ return NumberedShardSpec.class;
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java
new file mode 100644
index 0000000..97e8b9a
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java
@@ -0,0 +1,805 @@
+/*
+ * 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.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import it.unimi.dsi.fastutil.objects.AbstractObjectCollection;
+import it.unimi.dsi.fastutil.objects.ObjectCollection;
+import it.unimi.dsi.fastutil.objects.ObjectIterator;
+import it.unimi.dsi.fastutil.objects.ObjectIterators;
+import it.unimi.dsi.fastutil.objects.ObjectSortedSet;
+import it.unimi.dsi.fastutil.objects.ObjectSortedSets;
+import it.unimi.dsi.fastutil.shorts.AbstractShort2ObjectSortedMap;
+import it.unimi.dsi.fastutil.shorts.Short2ObjectMap;
+import it.unimi.dsi.fastutil.shorts.Short2ObjectRBTreeMap;
+import it.unimi.dsi.fastutil.shorts.Short2ObjectSortedMap;
+import it.unimi.dsi.fastutil.shorts.ShortComparator;
+import it.unimi.dsi.fastutil.shorts.ShortComparators;
+import it.unimi.dsi.fastutil.shorts.ShortSortedSet;
+import it.unimi.dsi.fastutil.shorts.ShortSortedSets;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.timeline.Overshadowable;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.TreeMap;
+
+/**
+ * OvershadowableManager manages the state of {@link AtomicUpdateGroup}. See the below {@link State} for details of
+ * the possible state.
+ * Note that an AtomicUpdateGroup can consist of {@link Overshadowable}s of the same majorVersion, minorVersion,
+ * rootPartition range, and atomicUpdateGroupSize.
+ * In {@link org.apache.druid.timeline.VersionedIntervalTimeline}, this class is used to manage segments in the same
+ * timeChunk.
+ *
+ * This class is not thread-safe.
+ */
+class OvershadowableManager<T extends Overshadowable<T>>
+{
+ private enum State
+ {
+ STANDBY, // have atomicUpdateGroup of higher versions than visible
+ VISIBLE, // have a single fully available atomicUpdateGroup of highest version
+ OVERSHADOWED // have atomicUpdateGroup of lower versions than visible
+ }
+
+ private final Map<Integer, PartitionChunk<T>> knownPartitionChunks; // served segments
+
+ // (start partitionId, end partitionId) -> minorVersion -> atomicUpdateGroup
+ private final TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> standbyGroups;
+ private final TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> visibleGroup;
+ private final TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> overshadowedGroups;
+
+ OvershadowableManager()
+ {
+ this.knownPartitionChunks = new HashMap<>();
+ this.standbyGroups = new TreeMap<>();
+ this.visibleGroup = new TreeMap<>();
+ this.overshadowedGroups = new TreeMap<>();
+ }
+
+ OvershadowableManager(OvershadowableManager<T> other)
+ {
+ this.knownPartitionChunks = new HashMap<>(other.knownPartitionChunks);
+ this.standbyGroups = new TreeMap<>(other.standbyGroups);
+ this.visibleGroup = new TreeMap<>(other.visibleGroup);
+ this.overshadowedGroups = new TreeMap<>(other.overshadowedGroups);
+ }
+
+ private TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> getStateMap(State state)
+ {
+ switch (state) {
+ case STANDBY:
+ return standbyGroups;
+ case VISIBLE:
+ return visibleGroup;
+ case OVERSHADOWED:
+ return overshadowedGroups;
+ default:
+ throw new ISE("Unknown state[%s]", state);
+ }
+ }
+
+ private Short2ObjectSortedMap<AtomicUpdateGroup<T>> createMinorVersionToAugMap(State state)
+ {
+ switch (state) {
+ case STANDBY:
+ case OVERSHADOWED:
+ return new Short2ObjectRBTreeMap<>();
+ case VISIBLE:
+ return new SingleEntryShort2ObjectSortedMap<>();
+ default:
+ throw new ISE("Unknown state[%s]", state);
+ }
+ }
+
+ private void transitAtomicUpdateGroupState(AtomicUpdateGroup<T> atomicUpdateGroup, State from, State to)
+ {
+ Preconditions.checkNotNull(atomicUpdateGroup, "atomicUpdateGroup");
+ Preconditions.checkArgument(!atomicUpdateGroup.isEmpty(), "empty atomicUpdateGroup");
+
+ removeFrom(atomicUpdateGroup, from);
+ addAtomicUpdateGroupWithState(atomicUpdateGroup, to);
+ }
+
+ /**
+ * Find the {@link AtomicUpdateGroup} of the given state which has the same {@link RootPartitionRange} and
+ * minorVersion with {@link PartitionChunk}.
+ */
+ @Nullable
+ private AtomicUpdateGroup<T> findAtomicUpdateGroupWith(PartitionChunk<T> chunk, State state)
+ {
+ final Short2ObjectSortedMap<AtomicUpdateGroup<T>> versionToGroup = getStateMap(state).get(
+ RootPartitionRange.of(chunk)
+ );
+ if (versionToGroup != null) {
+ final AtomicUpdateGroup<T> atomicUpdateGroup = versionToGroup.get(chunk.getObject().getMinorVersion());
+ if (atomicUpdateGroup != null) {
+ return atomicUpdateGroup;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Returns null if atomicUpdateGroup is not found for the state.
+ * Can return an empty atomicUpdateGroup.
+ */
+ @Nullable
+ private AtomicUpdateGroup<T> tryRemoveChunkFromGroupWithState(PartitionChunk<T> chunk, State state)
+ {
+ final RootPartitionRange rangeKey = RootPartitionRange.of(chunk);
+ final Short2ObjectSortedMap<AtomicUpdateGroup<T>> versionToGroup = getStateMap(state).get(rangeKey);
+ if (versionToGroup != null) {
+ final AtomicUpdateGroup<T> atomicUpdateGroup = versionToGroup.get(chunk.getObject().getMinorVersion());
+ if (atomicUpdateGroup != null) {
+ atomicUpdateGroup.remove(chunk);
+ if (atomicUpdateGroup.isEmpty()) {
+ versionToGroup.remove(chunk.getObject().getMinorVersion());
+ if (versionToGroup.isEmpty()) {
+ getStateMap(state).remove(rangeKey);
+ }
+ }
+
+ determineVisibleGroupAfterRemove(
+ atomicUpdateGroup,
+ RootPartitionRange.of(chunk),
+ chunk.getObject().getMinorVersion(),
+ state
+ );
+ return atomicUpdateGroup;
+ }
+ }
+ return null;
+ }
+
+ private List<Short2ObjectMap.Entry<AtomicUpdateGroup<T>>> findOvershadowedBy(
+ AtomicUpdateGroup<T> aug,
+ State fromState
+ )
+ {
+ final RootPartitionRange rangeKeyOfGivenAug = RootPartitionRange.of(aug);
+ return findOvershadowedBy(rangeKeyOfGivenAug, aug.getMinorVersion(), fromState);
+ }
+
+ /**
+ * Find all atomicUpdateGroups of the given state overshadowed by the given rootPartitionRange and minorVersion.
+ * The atomicUpdateGroup of a higher minorVersion can have a wider RootPartitionRange.
+ * To find all atomicUpdateGroups overshadowed by the given rootPartitionRange and minorVersion,
+ * we first need to find the first key contained by the given rootPartitionRange.
+ * Once we find such key, then we go through the entire map until we see an atomicUpdateGroup of which
+ * rootRangePartition is not contained by the given rootPartitionRange.
+ */
+ private List<Short2ObjectMap.Entry<AtomicUpdateGroup<T>>> findOvershadowedBy(
+ RootPartitionRange rangeOfAug,
+ short minorVersion,
+ State fromState
+ )
+ {
+ final TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> stateMap = getStateMap(fromState);
+ Entry<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> current = stateMap.floorEntry(rangeOfAug);
+
+ if (current == null) {
+ return Collections.emptyList();
+ }
+
+ // Find the first key for searching for overshadowed atomicUpdateGroup
+ while (true) {
+ final Entry<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> lowerEntry = stateMap.lowerEntry(
+ current.getKey()
+ );
+ if (lowerEntry != null && lowerEntry.getKey().startPartitionId == rangeOfAug.startPartitionId) {
+ current = lowerEntry;
+ } else {
+ break;
+ }
+ }
+
+ // Going through the map to find all entries of the RootPartitionRange contained by the given rangeOfAug.
+ // Note that RootPartitionRange of entries are always consecutive.
+ final List<Short2ObjectMap.Entry<AtomicUpdateGroup<T>>> found = new ArrayList<>();
+ while (current != null && rangeOfAug.contains(current.getKey())) {
+ // versionToGroup is sorted by minorVersion.
+ // versionToGroup.subMap(firstKey, minorVersion) below returns a map containing all entries of lower minorVersions
+ // than the given minorVersion.
+ final Short2ObjectSortedMap<AtomicUpdateGroup<T>> versionToGroup = current.getValue();
+ // Short2ObjectRBTreeMap.SubMap.short2ObjectEntrySet() implementation, especially size(), is not optimized.
+ // Note that size() is indirectly called in ArrayList.addAll() when ObjectSortedSet.toArray() is called.
+ // See AbstractObjectCollection.toArray().
+ // If you see performance degradation here, probably we need to improve the below line.
+ found.addAll(versionToGroup.subMap(versionToGroup.firstShortKey(), minorVersion).short2ObjectEntrySet());
+ current = stateMap.higherEntry(current.getKey());
+ }
+ return found;
+ }
+
+ /**
+ * Handles addition of the atomicUpdateGroup to the given state
+ */
+ private void transitionStandbyGroupIfFull(AtomicUpdateGroup<T> aug, State stateOfAug)
+ {
+ if (stateOfAug == State.STANDBY) {
+ // A standby atomicUpdateGroup becomes visible when its all segments are available.
+ if (aug.isFull()) {
+ // A visible atomicUpdateGroup becomes overshadowed when a fully available standby atomicUpdateGroup becomes
+ // visible which overshadows the current visible one.
+ findOvershadowedBy(aug, State.VISIBLE)
+ .forEach(entry -> transitAtomicUpdateGroupState(entry.getValue(), State.VISIBLE, State.OVERSHADOWED));
+ findOvershadowedBy(aug, State.STANDBY)
+ .forEach(entry -> transitAtomicUpdateGroupState(entry.getValue(), State.STANDBY, State.OVERSHADOWED));
+ transitAtomicUpdateGroupState(aug, State.STANDBY, State.VISIBLE);
+ }
+ }
+ }
+
+ private void addAtomicUpdateGroupWithState(AtomicUpdateGroup<T> aug, State state)
+ {
+ final AtomicUpdateGroup<T> existing = getStateMap(state)
+ .computeIfAbsent(RootPartitionRange.of(aug), k -> createMinorVersionToAugMap(state))
+ .put(aug.getMinorVersion(), aug);
+
+ if (existing != null) {
+ throw new ISE("AtomicUpdateGroup[%s] is already in state[%s]", aug, state);
+ }
+
+ transitionStandbyGroupIfFull(aug, state);
+ }
+
+ public boolean addChunk(PartitionChunk<T> chunk)
+ {
+ // Sanity check. ExistingChunk should be usually null.
+ final PartitionChunk<T> existingChunk = knownPartitionChunks.put(chunk.getChunkNumber(), chunk);
+ if (existingChunk != null) {
+ if (!existingChunk.equals(chunk)) {
+ throw new ISE(
+ "existingChunk[%s] is different from newChunk[%s] for partitionId[%d]",
+ existingChunk,
+ chunk,
+ chunk.getChunkNumber()
+ );
+ } else {
+ // A new chunk of the same major version and partitionId can be added in segment handoff
+ // from stream ingestion tasks to historicals
+ return false;
+ }
+ }
+
+ // Find atomicUpdateGroup of the new chunk
+ AtomicUpdateGroup<T> atomicUpdateGroup = findAtomicUpdateGroupWith(chunk, State.OVERSHADOWED);
+
+ if (atomicUpdateGroup != null) {
+ atomicUpdateGroup.add(chunk);
+ } else {
+ atomicUpdateGroup = findAtomicUpdateGroupWith(chunk, State.STANDBY);
+
+ if (atomicUpdateGroup != null) {
+ atomicUpdateGroup.add(chunk);
+ transitionStandbyGroupIfFull(atomicUpdateGroup, State.STANDBY);
+ } else {
+ atomicUpdateGroup = findAtomicUpdateGroupWith(chunk, State.VISIBLE);
+
+ if (atomicUpdateGroup != null) {
+ atomicUpdateGroup.add(chunk);
+ } else {
+ final AtomicUpdateGroup<T> newAtomicUpdateGroup = new AtomicUpdateGroup<>(chunk);
+
+ // Decide the initial state of the new atomicUpdateGroup
+ final boolean overshadowed = visibleGroup
+ .values()
+ .stream()
+ .flatMap(map -> map.values().stream())
+ .anyMatch(group -> group.overshadows(newAtomicUpdateGroup));
+
+ if (overshadowed) {
+ addAtomicUpdateGroupWithState(newAtomicUpdateGroup, State.OVERSHADOWED);
+ } else {
+ addAtomicUpdateGroupWithState(newAtomicUpdateGroup, State.STANDBY);
+ }
+ }
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Handles of removal of an empty atomicUpdateGroup from a state.
+ */
+ private void determineVisibleGroupAfterRemove(
+ AtomicUpdateGroup<T> augOfRemovedChunk,
+ RootPartitionRange rangeOfAug,
+ short minorVersion,
+ State stateOfRemovedAug
+ )
+ {
+ // If an atomicUpdateGroup is overshadowed by another non-visible atomicUpdateGroup, there must be another visible
+ // atomicUpdateGroup which also overshadows the same atomicUpdateGroup.
+ // As a result, the state of overshadowed atomicUpdateGroup should be updated only when a visible atomicUpdateGroup
+ // is removed.
+
+ if (stateOfRemovedAug == State.VISIBLE) {
+ // All segments in the visible atomicUpdateGroup which overshadows this atomicUpdateGroup is removed.
+ // Fall back if there is a fully available overshadowed atomicUpdateGroup
+
+ final List<AtomicUpdateGroup<T>> latestFullAugs = findLatestFullyAvailableOvershadowedAtomicUpdateGroup(
+ rangeOfAug,
+ minorVersion
+ );
+
+ // If there is no fully available fallback group, then the existing VISIBLE group remains VISIBLE.
+ // Otherwise, the latest fully available group becomes VISIBLE.
+ if (!latestFullAugs.isEmpty()) {
+ // Move the atomicUpdateGroup to standby
+ // and move the fully available overshadowed atomicUpdateGroup to visible
+ if (!augOfRemovedChunk.isEmpty()) {
+ transitAtomicUpdateGroupState(augOfRemovedChunk, State.VISIBLE, State.STANDBY);
+ }
+ latestFullAugs.forEach(group -> transitAtomicUpdateGroupState(group, State.OVERSHADOWED, State.VISIBLE));
+ }
+ }
+ }
+
+ private List<AtomicUpdateGroup<T>> findLatestFullyAvailableOvershadowedAtomicUpdateGroup(
+ RootPartitionRange rangeOfAug,
+ short minorVersion
+ )
+ {
+ final List<Short2ObjectMap.Entry<AtomicUpdateGroup<T>>> overshadowedGroups = findOvershadowedBy(
+ rangeOfAug,
+ minorVersion,
+ State.OVERSHADOWED
+ );
+ if (overshadowedGroups.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ final OvershadowableManager<T> manager = new OvershadowableManager<>();
+ for (Short2ObjectMap.Entry<AtomicUpdateGroup<T>> entry : overshadowedGroups) {
+ for (PartitionChunk<T> chunk : entry.getValue().getChunks()) {
+ manager.addChunk(chunk);
+ }
+ }
+
+ final List<AtomicUpdateGroup<T>> visibles = new ArrayList<>();
+ for (Short2ObjectSortedMap<AtomicUpdateGroup<T>> map : manager.visibleGroup.values()) {
+ visibles.addAll(map.values());
+ }
+ return visibles;
+ }
+
+ private void removeFrom(AtomicUpdateGroup<T> aug, State state)
+ {
+ final RootPartitionRange rangeKey = RootPartitionRange.of(aug);
+ final Short2ObjectSortedMap<AtomicUpdateGroup<T>> versionToGroup = getStateMap(state).get(rangeKey);
+ if (versionToGroup == null) {
+ throw new ISE("Unknown atomicUpdateGroup[%s] in state[%s]", aug, state);
+ }
+
+ final AtomicUpdateGroup<T> removed = versionToGroup.remove(aug.getMinorVersion());
+ if (removed == null) {
+ throw new ISE("Unknown atomicUpdateGroup[%s] in state[%s]", aug, state);
+ }
+
+ if (!removed.equals(aug)) {
+ throw new ISE(
+ "WTH? actually removed atomicUpdateGroup[%s] is different from the one which is supposed to be[%s]",
+ removed,
+ aug
+ );
+ }
+
+ if (versionToGroup.isEmpty()) {
+ getStateMap(state).remove(rangeKey);
+ }
+ }
+
+ @Nullable
+ public PartitionChunk<T> removeChunk(PartitionChunk<T> partitionChunk)
+ {
+ final PartitionChunk<T> knownChunk = knownPartitionChunks.get(partitionChunk.getChunkNumber());
+ if (knownChunk == null) {
+ return null;
+ }
+
+ if (!knownChunk.equals(partitionChunk)) {
+ throw new ISE(
+ "WTH? Same partitionId[%d], but known partition[%s] is different from the input partition[%s]",
+ partitionChunk.getChunkNumber(),
+ knownChunk,
+ partitionChunk
+ );
+ }
+
+ AtomicUpdateGroup<T> augOfRemovedChunk = tryRemoveChunkFromGroupWithState(partitionChunk, State.STANDBY);
+
+ if (augOfRemovedChunk == null) {
+ augOfRemovedChunk = tryRemoveChunkFromGroupWithState(partitionChunk, State.VISIBLE);
+ if (augOfRemovedChunk == null) {
+ augOfRemovedChunk = tryRemoveChunkFromGroupWithState(partitionChunk, State.OVERSHADOWED);
+ if (augOfRemovedChunk == null) {
+ throw new ISE("Can't find atomicUpdateGroup for partitionChunk[%s]", partitionChunk);
+ }
+ }
+ }
+
+ return knownPartitionChunks.remove(partitionChunk.getChunkNumber());
+ }
+
+ public boolean isEmpty()
+ {
+ return visibleGroup.isEmpty();
+ }
+
+ public boolean isComplete()
+ {
+ return visibleGroup.values().stream().allMatch(map -> Iterables.getOnlyElement(map.values()).isFull());
+ }
+
+ @Nullable
+ public PartitionChunk<T> getChunk(int partitionId)
+ {
+ final PartitionChunk<T> chunk = knownPartitionChunks.get(partitionId);
+ if (chunk == null) {
+ return null;
+ }
+ final AtomicUpdateGroup<T> aug = findAtomicUpdateGroupWith(chunk, State.VISIBLE);
+ if (aug == null) {
+ return null;
+ } else {
+ return Preconditions.checkNotNull(
+ aug.findChunk(partitionId),
+ "Can't find partitionChunk for partitionId[%s] in atomicUpdateGroup[%s]",
+ partitionId,
+ aug
+ );
+ }
+ }
+
+ public List<PartitionChunk<T>> getVisibles()
+ {
+ final List<PartitionChunk<T>> visibles = new ArrayList<>();
+ for (Short2ObjectSortedMap<AtomicUpdateGroup<T>> treeMap : visibleGroup.values()) {
+ for (AtomicUpdateGroup<T> aug : treeMap.values()) {
+ visibles.addAll(aug.getChunks());
+ }
+ }
+ return visibles;
+ }
+
+ public List<PartitionChunk<T>> getOvershadowed()
+ {
+ final List<PartitionChunk<T>> overshadowed = new ArrayList<>();
+ for (Short2ObjectSortedMap<AtomicUpdateGroup<T>> treeMap : overshadowedGroups.values()) {
+ for (AtomicUpdateGroup<T> aug : treeMap.values()) {
+ overshadowed.addAll(aug.getChunks());
+ }
+ }
+ return overshadowed;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ OvershadowableManager<?> that = (OvershadowableManager<?>) o;
+ return Objects.equals(knownPartitionChunks, that.knownPartitionChunks) &&
+ Objects.equals(standbyGroups, that.standbyGroups) &&
+ Objects.equals(visibleGroup, that.visibleGroup) &&
+ Objects.equals(overshadowedGroups, that.overshadowedGroups);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(knownPartitionChunks, standbyGroups, visibleGroup, overshadowedGroups);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "OvershadowableManager{" +
+ "knownPartitionChunks=" + knownPartitionChunks +
+ ", standbyGroups=" + standbyGroups +
+ ", visibleGroup=" + visibleGroup +
+ ", overshadowedGroups=" + overshadowedGroups +
+ '}';
+ }
+
+ private static class RootPartitionRange implements Comparable<RootPartitionRange>
+ {
+ private final short startPartitionId;
+ private final short endPartitionId;
+
+ private static <T extends Overshadowable<T>> RootPartitionRange of(PartitionChunk<T> chunk)
+ {
+ return of(chunk.getObject().getStartRootPartitionId(), chunk.getObject().getEndRootPartitionId());
+ }
+
+ private static <T extends Overshadowable<T>> RootPartitionRange of(AtomicUpdateGroup<T> aug)
+ {
+ return of(aug.getStartRootPartitionId(), aug.getEndRootPartitionId());
+ }
+
+ private static RootPartitionRange of(int startPartitionId, int endPartitionId)
+ {
+ return new RootPartitionRange((short) startPartitionId, (short) endPartitionId);
+ }
+
+ private RootPartitionRange(short startPartitionId, short endPartitionId)
+ {
+ this.startPartitionId = startPartitionId;
+ this.endPartitionId = endPartitionId;
+ }
+
+ public boolean contains(RootPartitionRange that)
+ {
+ return this.startPartitionId <= that.startPartitionId && this.endPartitionId >= that.endPartitionId;
+ }
+
+ @Override
+ public int compareTo(RootPartitionRange o)
+ {
+ if (startPartitionId != o.startPartitionId) {
+ return Integer.compare(Short.toUnsignedInt(startPartitionId), Short.toUnsignedInt(o.startPartitionId));
+ } else {
+ return Integer.compare(Short.toUnsignedInt(endPartitionId), Short.toUnsignedInt(o.endPartitionId));
+ }
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ RootPartitionRange that = (RootPartitionRange) o;
+ return startPartitionId == that.startPartitionId &&
+ endPartitionId == that.endPartitionId;
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(startPartitionId, endPartitionId);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "RootPartitionRange{" +
+ "startPartitionId=" + startPartitionId +
+ ", endPartitionId=" + endPartitionId +
+ '}';
+ }
+ }
+
+ /**
+ * Map can store at most a single entry.
+ * Comparing to{@link it.unimi.dsi.fastutil.shorts.Short2ObjectSortedMaps.Singleton}, it's different from the
+ * perspective of that this class supports update.
+ */
+ private static class SingleEntryShort2ObjectSortedMap<V> extends AbstractShort2ObjectSortedMap<V>
+ {
+ private short key;
+ private V val;
+
+ private SingleEntryShort2ObjectSortedMap()
+ {
+ key = -1;
+ val = null;
+ }
+
+ @Override
+ public Short2ObjectSortedMap<V> subMap(short fromKey, short toKey)
+ {
+ if (fromKey <= key && toKey > key) {
+ return this;
+ } else {
+ throw new IAE("fromKey: %s, toKey: %s, key: %s", fromKey, toKey, key);
+ }
+ }
+
+ @Override
+ public Short2ObjectSortedMap<V> headMap(short toKey)
+ {
+ if (toKey > key) {
+ return this;
+ } else {
+ throw new IllegalArgumentException();
+ }
+ }
+
+ @Override
+ public Short2ObjectSortedMap<V> tailMap(short fromKey)
+ {
+ if (fromKey <= key) {
+ return this;
+ } else {
+ throw new IllegalArgumentException();
+ }
+ }
+
+ @Override
+ public short firstShortKey()
+ {
+ if (key < 0) {
+ throw new NoSuchElementException();
+ }
+ return key;
+ }
+
+ @Override
+ public short lastShortKey()
+ {
+ if (key < 0) {
+ throw new NoSuchElementException();
+ }
+ return key;
+ }
+
+ @Override
+ public ObjectSortedSet<Short2ObjectMap.Entry<V>> short2ObjectEntrySet()
+ {
+ return isEmpty() ? ObjectSortedSets.EMPTY_SET : ObjectSortedSets.singleton(new BasicEntry<>(key, val));
+ }
+
+ @Override
+ public ShortSortedSet keySet()
+ {
+ return isEmpty() ? ShortSortedSets.EMPTY_SET : ShortSortedSets.singleton(key);
+ }
+
+ @Override
+ public ObjectCollection<V> values()
+ {
+ return new AbstractObjectCollection<V>()
+ {
+ @Override
+ public ObjectIterator<V> iterator()
+ {
+ return size() > 0 ? ObjectIterators.singleton(val) : ObjectIterators.emptyIterator();
+ }
+
+ @Override
+ public int size()
+ {
+ return key < 0 ? 0 : 1;
+ }
+ };
+ }
+
+ @Override
+ public V put(final short key, final V value)
+ {
+ if (isEmpty()) {
+ this.key = key;
+ this.val = value;
+ return null;
+ } else {
+ if (this.key == key) {
+ final V existing = this.val;
+ this.val = value;
+ return existing;
+ } else {
+ throw new ISE(
+ "Can't add [%d, %s] to non-empty SingleEntryShort2ObjectSortedMap[%d, %s]",
+ key,
+ value,
+ this.key,
+ this.val
+ );
+ }
+ }
+ }
+
+ @Override
+ public V get(short key)
+ {
+ return this.key == key ? val : null;
+ }
+
+ @Override
+ public V remove(final short key)
+ {
+ if (this.key == key) {
+ this.key = -1;
+ return val;
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public boolean containsKey(short key)
+ {
+ return this.key == key;
+ }
+
+ @Override
+ public ShortComparator comparator()
+ {
+ return ShortComparators.NATURAL_COMPARATOR;
+ }
+
+ @Override
+ public int size()
+ {
+ return key < 0 ? 0 : 1;
+ }
+
+ @Override
+ public void defaultReturnValue(V rv)
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public V defaultReturnValue()
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean isEmpty()
+ {
+ return key < 0;
+ }
+
+ @Override
+ public boolean containsValue(Object value)
+ {
+ if (key < 0) {
+ return false;
+ } else {
+ return Objects.equals(val, value);
+ }
+ }
+
+ @Override
+ public void putAll(Map<? extends Short, ? extends V> m)
+ {
+ if (!m.isEmpty()) {
+ if (m.size() == 1) {
+ final Map.Entry<? extends Short, ? extends V> entry = m.entrySet().iterator().next();
+ this.key = entry.getKey();
+ this.val = entry.getValue();
+ } else {
+ throw new IllegalArgumentException();
+ }
+ }
+ }
+ }
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java
similarity index 61%
copy from core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
copy to core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java
index 5003f65..0fea564 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java
@@ -20,23 +20,18 @@
package org.apache.druid.timeline.partition;
/**
+ * ShardSpec for non-first-generation segments.
+ * This shardSpec is allocated a partitionId between {@link PartitionIds#NON_ROOT_GEN_START_PARTITION_ID} and
+ * {@link PartitionIds#NON_ROOT_GEN_END_PARTITION_ID}.
+ *
+ * @see org.apache.druid.timeline.Overshadowable
*/
-public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
+public interface OverwriteShardSpec extends ShardSpec
{
- public ImmutablePartitionHolder(PartitionHolder<T> partitionHolder)
- {
- super(partitionHolder);
- }
-
- @Override
- public PartitionChunk<T> remove(PartitionChunk<T> tPartitionChunk)
+ default OverwriteShardSpec withAtomicUpdateGroupSize(int atomicUpdateGroupSize)
{
- throw new UnsupportedOperationException();
+ return withAtomicUpdateGroupSize((short) atomicUpdateGroupSize);
}
- @Override
- public boolean add(PartitionChunk<T> tPartitionChunk)
- {
- throw new UnsupportedOperationException();
- }
+ OverwriteShardSpec withAtomicUpdateGroupSize(short atomicUpdateGroupSize);
}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java
index dcf29ae..26e34da 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java
@@ -20,30 +20,32 @@
package org.apache.druid.timeline.partition;
import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
+import org.apache.druid.timeline.Overshadowable;
import javax.annotation.Nullable;
import java.util.Iterator;
import java.util.List;
+import java.util.Objects;
import java.util.Spliterator;
-import java.util.TreeMap;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
/**
* An object that clumps together multiple other objects which each represent a shard of some space.
*/
-public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
+public class PartitionHolder<T extends Overshadowable<T>> implements Iterable<PartitionChunk<T>>
{
- private final TreeMap<PartitionChunk<T>, PartitionChunk<T>> holderMap;
+ private final OvershadowableManager<T> overshadowableManager;
public PartitionHolder(PartitionChunk<T> initialChunk)
{
- this.holderMap = new TreeMap<>();
+ this.overshadowableManager = new OvershadowableManager<>();
add(initialChunk);
}
public PartitionHolder(List<PartitionChunk<T>> initialChunks)
{
- this.holderMap = new TreeMap<>();
+ this.overshadowableManager = new OvershadowableManager<>();
for (PartitionChunk<T> chunk : initialChunks) {
add(chunk);
}
@@ -51,33 +53,32 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
public PartitionHolder(PartitionHolder<T> partitionHolder)
{
- this.holderMap = new TreeMap<>();
- this.holderMap.putAll(partitionHolder.holderMap);
+ this.overshadowableManager = new OvershadowableManager<>(partitionHolder.overshadowableManager);
}
public boolean add(PartitionChunk<T> chunk)
{
- return holderMap.putIfAbsent(chunk, chunk) == null;
+ return overshadowableManager.addChunk(chunk);
}
@Nullable
public PartitionChunk<T> remove(PartitionChunk<T> chunk)
{
- return holderMap.remove(chunk);
+ return overshadowableManager.removeChunk(chunk);
}
public boolean isEmpty()
{
- return holderMap.isEmpty();
+ return overshadowableManager.isEmpty();
}
public boolean isComplete()
{
- if (holderMap.isEmpty()) {
+ if (overshadowableManager.isEmpty()) {
return false;
}
- Iterator<PartitionChunk<T>> iter = holderMap.keySet().iterator();
+ Iterator<PartitionChunk<T>> iter = iterator();
PartitionChunk<T> curr = iter.next();
@@ -86,7 +87,7 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
}
if (curr.isEnd()) {
- return true;
+ return overshadowableManager.isComplete();
}
while (iter.hasNext()) {
@@ -96,7 +97,7 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
}
if (next.isEnd()) {
- return true;
+ return overshadowableManager.isComplete();
}
curr = next;
}
@@ -106,24 +107,29 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
public PartitionChunk<T> getChunk(final int partitionNum)
{
- final Iterator<PartitionChunk<T>> retVal = Iterators.filter(
- holderMap.keySet().iterator(),
- input -> input.getChunkNumber() == partitionNum
- );
-
- return retVal.hasNext() ? retVal.next() : null;
+ return overshadowableManager.getChunk(partitionNum);
}
@Override
public Iterator<PartitionChunk<T>> iterator()
{
- return holderMap.keySet().iterator();
+ return overshadowableManager.getVisibles().iterator();
}
@Override
public Spliterator<PartitionChunk<T>> spliterator()
{
- return holderMap.keySet().spliterator();
+ return overshadowableManager.getVisibles().spliterator();
+ }
+
+ public Stream<PartitionChunk<T>> stream()
+ {
+ return StreamSupport.stream(spliterator(), false);
+ }
+
+ public List<PartitionChunk<T>> getOvershadowed()
+ {
+ return overshadowableManager.getOvershadowed();
}
public Iterable<T> payloads()
@@ -140,27 +146,21 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
if (o == null || getClass() != o.getClass()) {
return false;
}
-
- PartitionHolder that = (PartitionHolder) o;
-
- if (!holderMap.equals(that.holderMap)) {
- return false;
- }
-
- return true;
+ PartitionHolder<?> that = (PartitionHolder<?>) o;
+ return Objects.equals(overshadowableManager, that.overshadowableManager);
}
@Override
public int hashCode()
{
- return holderMap.hashCode();
+ return Objects.hash(overshadowableManager);
}
@Override
public String toString()
{
return "PartitionHolder{" +
- "holderMap=" + holderMap +
+ "overshadowableManager=" + overshadowableManager +
'}';
}
}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java
similarity index 55%
copy from core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
copy to core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java
index 5003f65..fc5d0e9 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java
@@ -19,24 +19,28 @@
package org.apache.druid.timeline.partition;
-/**
- */
-public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
+public final class PartitionIds
{
- public ImmutablePartitionHolder(PartitionHolder<T> partitionHolder)
- {
- super(partitionHolder);
- }
+ /**
+ * Start partitionId available for root generation segments.
+ */
+ public static final int ROOT_GEN_START_PARTITION_ID = 0;
+ /**
+ * End partitionId available for root generation segments.
+ */
+ public static final int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive
+ /**
+ * Start partitionId available for non-root generation segments.
+ */
+ public static final int NON_ROOT_GEN_START_PARTITION_ID = 32768;
+ /**
+ * End partitionId available for non-root generation segments.
+ */
+ public static final int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive
- @Override
- public PartitionChunk<T> remove(PartitionChunk<T> tPartitionChunk)
- {
- throw new UnsupportedOperationException();
- }
+ public static final short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1;
- @Override
- public boolean add(PartitionChunk<T> tPartitionChunk)
+ private PartitionIds()
{
- throw new UnsupportedOperationException();
}
}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java
index 10a6d6c..43aaf70 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java
@@ -37,7 +37,8 @@ import java.util.Map;
@JsonSubTypes.Type(name = "single", value = SingleDimensionShardSpec.class),
@JsonSubTypes.Type(name = "linear", value = LinearShardSpec.class),
@JsonSubTypes.Type(name = "numbered", value = NumberedShardSpec.class),
- @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class)
+ @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class),
+ @JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwriteShardSpec.class)
})
public interface ShardSpec
{
@@ -47,6 +48,26 @@ public interface ShardSpec
int getPartitionNum();
+ default int getStartRootPartitionId()
+ {
+ return getPartitionNum();
+ }
+
+ default int getEndRootPartitionId()
+ {
+ return getPartitionNum() + 1;
+ }
+
+ default short getMinorVersion()
+ {
+ return 0;
+ }
+
+ default short getAtomicUpdateGroupSize()
+ {
+ return 1;
+ }
+
ShardSpecLookup getLookup(List<ShardSpec> shardSpecs);
/**
@@ -61,4 +82,9 @@ public interface ShardSpec
* @return possibility of in domain
*/
boolean possibleInDomain(Map<String, RangeSet<String>> domain);
+
+ /**
+ * Returns true if two segments of this and other shardSpecs can exist in the same timeChunk.
+ */
+ boolean isCompatible(Class<? extends ShardSpec> other);
}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java
new file mode 100644
index 0000000..7ab0be8
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java
@@ -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.timeline.partition;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import javax.annotation.Nullable;
+
+/**
+ * Factory to be used to allocate segments remotely in the overlord.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+@JsonSubTypes({
+ @JsonSubTypes.Type(name = "numbered", value = NumberedShardSpecFactory.class),
+ @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpecFactory.class),
+ @JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwritingShardSpecFactory.class),
+})
+public interface ShardSpecFactory
+{
+ /**
+ * Create a new shardSpec based on {@code specOfPreviousMaxPartitionId}. If it's null, it assumes that this is the
+ * first call for the timeChunk where the new segment is created.
+ * Note that {@code specOfPreviousMaxPartitionId} can also be null for {@link OverwriteShardSpec} if all segments
+ * in the timeChunk are first-generation segments.
+ */
+ ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId);
+
+ /**
+ * Create a new shardSpec having the given partitionId.
+ */
+ ShardSpec create(ObjectMapper objectMapper, int partitionId);
+
+ /**
+ * Return the class of the shardSpec created by this factory.
+ */
+ Class<? extends ShardSpec> getShardSpecClass();
+}
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java
index f2b8441..968a1d7 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java
@@ -33,9 +33,7 @@ import java.util.List;
import java.util.Map;
/**
- * Class uses getters/setters to work around http://jira.codehaus.org/browse/MSHADE-92
- *
- * Adjust to JsonCreator and final fields when resolved.
+ * {@link ShardSpec} for range partitioning based on a single dimension
*/
public class SingleDimensionShardSpec implements ShardSpec
{
@@ -46,6 +44,12 @@ public class SingleDimensionShardSpec implements ShardSpec
private final String end;
private final int partitionNum;
+ /**
+ * @param dimension partition dimension
+ * @param start inclusive start of this range
+ * @param end exclusive end of this range
+ * @param partitionNum unique ID for this shard
+ */
@JsonCreator
public SingleDimensionShardSpec(
@JsonProperty("dimension") String dimension,
@@ -54,6 +58,7 @@ public class SingleDimensionShardSpec implements ShardSpec
@JsonProperty("partitionNum") int partitionNum
)
{
+ Preconditions.checkArgument(partitionNum >= 0, "partitionNum >= 0");
this.dimension = Preconditions.checkNotNull(dimension, "dimension");
this.start = start;
this.end = end;
@@ -132,6 +137,12 @@ public class SingleDimensionShardSpec implements ShardSpec
}
@Override
+ public boolean isCompatible(Class<? extends ShardSpec> other)
+ {
+ return other == SingleDimensionShardSpec.class;
+ }
+
+ @Override
public <T> PartitionChunk<T> createChunk(T obj)
{
return new StringPartitionChunk<T>(start, end, partitionNum, obj);
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/SingleElementPartitionChunk.java b/core/src/main/java/org/apache/druid/timeline/partition/SingleElementPartitionChunk.java
index f12073f..2567fe6 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/SingleElementPartitionChunk.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/SingleElementPartitionChunk.java
@@ -93,7 +93,7 @@ public class SingleElementPartitionChunk<T> implements PartitionChunk<T>
@Override
public int hashCode()
{
- return element != null ? element.hashCode() : 0;
+ return 0;
}
@Override
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java b/core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java
index ff7171f..28d7505 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java
@@ -116,4 +116,15 @@ public class StringPartitionChunk<T> implements PartitionChunk<T>
result = 31 * result + (object != null ? object.hashCode() : 0);
return result;
}
+
+ @Override
+ public String toString()
+ {
+ return "StringPartitionChunk{" +
+ "start='" + start + '\'' +
+ ", end='" + end + '\'' +
+ ", chunkNumber=" + chunkNumber +
+ ", object=" + object +
+ '}';
+ }
}
diff --git a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java
index 7015785..bc3cf90 100644
--- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java
+++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java
@@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.timeline.partition.ShardSpecLookup;
@@ -92,6 +93,12 @@ public class DataSegmentTest
{
return true;
}
+
+ @Override
+ public boolean isCompatible(Class<? extends ShardSpec> other)
+ {
+ return false;
+ }
};
}
@@ -117,7 +124,7 @@ public class DataSegmentTest
loadSpec,
Arrays.asList("dim1", "dim2"),
Arrays.asList("met1", "met2"),
- NoneShardSpec.instance(),
+ new NumberedShardSpec(3, 0),
TEST_VERSION,
1
);
@@ -134,7 +141,7 @@ public class DataSegmentTest
Assert.assertEquals(loadSpec, objectMap.get("loadSpec"));
Assert.assertEquals("dim1,dim2", objectMap.get("dimensions"));
Assert.assertEquals("met1,met2", objectMap.get("metrics"));
- Assert.assertEquals(ImmutableMap.of("type", "none"), objectMap.get("shardSpec"));
+ Assert.assertEquals(ImmutableMap.of("type", "numbered", "partitionNum", 3, "partitions", 0), objectMap.get("shardSpec"));
Assert.assertEquals(TEST_VERSION, objectMap.get("binaryVersion"));
Assert.assertEquals(1, objectMap.get("size"));
diff --git a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
index 3e66bf5..6971833 100644
--- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
+++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
@@ -19,10 +19,10 @@
package org.apache.druid.timeline;
-import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import org.apache.druid.java.util.common.DateTimes;
@@ -30,8 +30,11 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.timeline.partition.ImmutablePartitionHolder;
import org.apache.druid.timeline.partition.IntegerPartitionChunk;
+import org.apache.druid.timeline.partition.NumberedOverwritingPartitionChunk;
+import org.apache.druid.timeline.partition.NumberedPartitionChunk;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.PartitionHolder;
+import org.apache.druid.timeline.partition.PartitionIds;
import org.apache.druid.timeline.partition.SingleElementPartitionChunk;
import org.joda.time.DateTime;
import org.joda.time.Days;
@@ -45,13 +48,14 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
/**
*/
public class VersionedIntervalTimelineTest
{
- VersionedIntervalTimeline<String, Integer> timeline;
+ VersionedIntervalTimeline<String, OvershadowableInteger> timeline;
@Before
public void setUp()
@@ -70,8 +74,8 @@ public class VersionedIntervalTimelineTest
add("2011-05-01/2011-05-10", "4", 9);
add("2011-10-01/2011-10-02", "1", 1);
- add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(null, 10, 0, 20));
- add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(10, null, 1, 21));
+ add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("3", 0, 20)));
+ add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger("3", 1, 21)));
add("2011-10-03/2011-10-04", "3", 3);
add("2011-10-04/2011-10-05", "4", 4);
add("2011-10-05/2011-10-06", "5", 5);
@@ -94,8 +98,8 @@ public class VersionedIntervalTimelineTest
public void testApril2()
{
Assert.assertEquals(
- makeSingle(1),
- timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle(1))
+ makeSingle("2", 1),
+ timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1))
);
assertValues(
Arrays.asList(
@@ -112,12 +116,12 @@ public class VersionedIntervalTimelineTest
public void testApril3()
{
Assert.assertEquals(
- makeSingle(1),
- timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle(1))
+ makeSingle("2", 1),
+ timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1))
);
Assert.assertEquals(
- makeSingle(2),
- timeline.remove(Intervals.of("2011-04-01/2011-04-03"), "1", makeSingle(2))
+ makeSingle("1", 2),
+ timeline.remove(Intervals.of("2011-04-01/2011-04-03"), "1", makeSingle("1", 2))
);
assertValues(
Arrays.asList(
@@ -133,8 +137,8 @@ public class VersionedIntervalTimelineTest
public void testApril4()
{
Assert.assertEquals(
- makeSingle(1),
- timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle(1))
+ makeSingle("2", 1),
+ timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1))
);
assertValues(
Arrays.asList(
@@ -168,7 +172,7 @@ public class VersionedIntervalTimelineTest
@Test
public void testMay2()
{
- Assert.assertNotNull(timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(1)));
+ Assert.assertNotNull(timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9)));
assertValues(
Arrays.asList(
createExpected("2011-05-01/2011-05-03", "2", 7),
@@ -183,12 +187,12 @@ public class VersionedIntervalTimelineTest
public void testMay3()
{
Assert.assertEquals(
- makeSingle(9),
- timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(9))
+ makeSingle("4", 9),
+ timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9))
);
Assert.assertEquals(
- makeSingle(7),
- timeline.remove(Intervals.of("2011-05-01/2011-05-05"), "2", makeSingle(7))
+ makeSingle("2", 7),
+ timeline.remove(Intervals.of("2011-05-01/2011-05-05"), "2", makeSingle("2", 7))
);
assertValues(
Arrays.asList(
@@ -227,35 +231,35 @@ public class VersionedIntervalTimelineTest
@Test
public void testRemove()
{
- for (TimelineObjectHolder<String, Integer> holder : timeline.findOvershadowed()) {
- for (PartitionChunk<Integer> chunk : holder.getObject()) {
+ for (TimelineObjectHolder<String, OvershadowableInteger> holder : timeline.findFullyOvershadowed()) {
+ for (PartitionChunk<OvershadowableInteger> chunk : holder.getObject()) {
timeline.remove(holder.getInterval(), holder.getVersion(), chunk);
}
}
- Assert.assertTrue(timeline.findOvershadowed().isEmpty());
+ Assert.assertTrue(timeline.findFullyOvershadowed().isEmpty());
}
@Test
public void testFindEntry()
{
Assert.assertEquals(
- new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
+ new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))),
timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1")
);
Assert.assertEquals(
- new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
+ new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))),
timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1")
);
Assert.assertEquals(
- new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
+ new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))),
timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1")
);
Assert.assertEquals(
- new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
+ new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))),
timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1")
);
@@ -279,7 +283,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-02/2011-01-05", "2", 1);
Assert.assertEquals(
- new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
+ new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))),
timeline.findEntry(Intervals.of("2011-01-02T02/2011-01-04"), "1")
);
}
@@ -293,8 +297,8 @@ public class VersionedIntervalTimelineTest
createExpected(
"2011-10-02/2011-10-03", "3",
Arrays.asList(
- IntegerPartitionChunk.make(null, 10, 0, 20),
- IntegerPartitionChunk.make(10, null, 1, 21)
+ IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("3", 0, 20)),
+ IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger("3", 1, 21))
)
),
createExpected("2011-10-03/2011-10-04", "3", 3),
@@ -310,36 +314,36 @@ public class VersionedIntervalTimelineTest
{
testRemove();
- add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, 60));
+ add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60)));
assertValues(
ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)),
timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
);
- Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
+ Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty());
- add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, 61));
+ add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61)));
assertValues(
ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)),
timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
);
- Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
+ Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty());
- add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, 62));
+ add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62)));
assertValues(
ImmutableList.of(
createExpected("2011-10-05/2011-10-06", "5", 5),
createExpected(
"2011-10-06/2011-10-07", "6",
Arrays.asList(
- IntegerPartitionChunk.make(null, 10, 0, 60),
- IntegerPartitionChunk.make(10, 20, 1, 61),
- IntegerPartitionChunk.make(20, null, 2, 62)
+ IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60)),
+ IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61)),
+ IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62))
)
)
),
timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
);
- Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
+ Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty());
}
@Test
@@ -347,18 +351,18 @@ public class VersionedIntervalTimelineTest
{
testRemove();
- add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, 60));
- Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
+ add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60)));
+ Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty());
- add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, 61));
- Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
+ add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61)));
+ Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty());
- add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, 62));
+ add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62)));
assertValues(
ImmutableSet.of(
createExpected("2011-10-05/2011-10-06", "5", 5)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -367,13 +371,18 @@ public class VersionedIntervalTimelineTest
{
testIncompletePartitionDoesNotOvershadow();
- final IntegerPartitionChunk<Integer> chunk = IntegerPartitionChunk.make(null, 10, 0, 60);
+ final IntegerPartitionChunk<OvershadowableInteger> chunk = IntegerPartitionChunk.make(
+ null,
+ 10,
+ 0,
+ new OvershadowableInteger("6", 0, 60)
+ );
Assert.assertEquals(chunk, timeline.remove(Intervals.of("2011-10-05/2011-10-07"), "6", chunk));
assertValues(
ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)),
timeline.lookup(Intervals.of("2011-10-05/2011-10-07"))
);
- Assert.assertTrue("Expected no overshadowed entries", timeline.findOvershadowed().isEmpty());
+ Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty());
}
@Test
@@ -388,8 +397,8 @@ public class VersionedIntervalTimelineTest
);
Assert.assertEquals(
- makeSingle(10),
- timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "5", makeSingle(10))
+ makeSingle("5", 10),
+ timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "5", makeSingle("5", 10))
);
assertValues(
Collections.singletonList(
@@ -407,8 +416,8 @@ public class VersionedIntervalTimelineTest
);
Assert.assertEquals(
- makeSingle(9),
- timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(9))
+ makeSingle("4", 9),
+ timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9))
);
assertValues(
Collections.singletonList(
@@ -429,6 +438,8 @@ public class VersionedIntervalTimelineTest
add("2011-01-05/2011-01-15", "1", 3);
}
+ // 2|----|
+ // 2|----|
// 1|----|
// 1|----|
@Test
@@ -560,8 +571,8 @@ public class VersionedIntervalTimelineTest
assertValues(
Arrays.asList(
- createExpected("2011-01-01/2011-01-10", "2", 1),
- createExpected("2011-01-10/2011-01-15", "1", 3)
+ createExpected("2011-01-01/2011-01-10", "2", 3),
+ createExpected("2011-01-10/2011-01-15", "1", 1)
),
timeline.lookup(Intervals.of("2011-01-01/2011-01-15"))
);
@@ -930,7 +941,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-01/2011-01-03", "1", 1),
createExpected("2011-01-03/2011-01-05", "2", 2),
createExpected("2011-01-05/2011-01-13", "1", 1),
- createExpected("2011-01-13/2011-01-20", "2", 2)
+ createExpected("2011-01-13/2011-01-20", "2", 3)
),
timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
);
@@ -949,8 +960,8 @@ public class VersionedIntervalTimelineTest
assertValues(
Arrays.asList(
- createExpected("2011-01-01/2011-01-15", "2", 2),
- createExpected("2011-01-15/2011-01-20", "1", 1)
+ createExpected("2011-01-01/2011-01-15", "2", 1),
+ createExpected("2011-01-15/2011-01-20", "1", 3)
),
timeline.lookup(Intervals.of("2011-01-01/2011-01-20"))
);
@@ -1109,7 +1120,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-03/2011-01-06", "1", 1),
createExpected("2011-01-09/2011-01-12", "1", 2)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1132,7 +1143,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-05/2011-01-10", "2", 2),
createExpected("2011-01-01/2011-01-10", "1", 3)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1154,7 +1165,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-01-03/2011-01-12", "1", 3),
createExpected("2011-01-01/2011-01-05", "2", 1)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1166,7 +1177,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-01/2011-01-20", "1", 1);
add("2011-01-10/2011-01-15", "2", 2);
- timeline.remove(Intervals.of("2011-01-10/2011-01-15"), "2", makeSingle(2));
+ timeline.remove(Intervals.of("2011-01-10/2011-01-15"), "2", makeSingle("2", 2));
assertValues(
Collections.singletonList(
@@ -1185,7 +1196,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-10/2011-01-20", "2", 2);
add("2011-01-20/2011-01-30", "3", 4);
- timeline.remove(Intervals.of("2011-01-10/2011-01-20"), "2", makeSingle(2));
+ timeline.remove(Intervals.of("2011-01-10/2011-01-20"), "2", makeSingle("2", 2));
assertValues(
Arrays.asList(
@@ -1206,8 +1217,8 @@ public class VersionedIntervalTimelineTest
add("2011-01-02/2011-01-03", "2", 2);
add("2011-01-10/2011-01-14", "2", 3);
- timeline.remove(Intervals.of("2011-01-02/2011-01-03"), "2", makeSingle(2));
- timeline.remove(Intervals.of("2011-01-10/2011-01-14"), "2", makeSingle(3));
+ timeline.remove(Intervals.of("2011-01-02/2011-01-03"), "2", makeSingle("2", 2));
+ timeline.remove(Intervals.of("2011-01-10/2011-01-14"), "2", makeSingle("2", 3));
assertValues(
Collections.singletonList(
@@ -1227,7 +1238,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-10/2011-01-15", "2", 2);
add("2011-01-15/2011-01-20", "2", 3);
- timeline.remove(Intervals.of("2011-01-15/2011-01-20"), "2", makeSingle(3));
+ timeline.remove(Intervals.of("2011-01-15/2011-01-20"), "2", makeSingle("2", 3));
assertValues(
Arrays.asList(
@@ -1246,7 +1257,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-01/2011-01-20", "1", 1);
add("2011-01-10/2011-01-15", "2", 2);
- timeline.remove(Intervals.of("2011-01-10/2011-01-15"), "2", makeSingle(2));
+ timeline.remove(Intervals.of("2011-01-10/2011-01-15"), "2", makeSingle("2", 2));
add("2011-01-01/2011-01-20", "1", 1);
assertValues(
@@ -1262,11 +1273,11 @@ public class VersionedIntervalTimelineTest
{
Assert.assertNull(
"Don't have it, should be null",
- timeline.remove(Intervals.of("1970-01-01/2025-04-20"), "1", makeSingle(1))
+ timeline.remove(Intervals.of("1970-01-01/2025-04-20"), "1", makeSingle("1", 1))
);
Assert.assertNull(
"Don't have it, should be null",
- timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "version does not exist", makeSingle(1))
+ timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "version does not exist", makeSingle("version does not exist", 1))
);
}
@@ -1280,7 +1291,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-10/2011-01-15", "3", 3);
add("2011-01-15/2011-01-20", "4", 4);
- timeline.remove(Intervals.of("2011-01-15/2011-01-20"), "4", makeSingle(4));
+ timeline.remove(Intervals.of("2011-01-15/2011-01-20"), "4", makeSingle("4", 4));
assertValues(
Arrays.asList(
@@ -1308,7 +1319,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3),
createExpected("2011-04-06/2011-04-09", "1", 4)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1326,7 +1337,7 @@ public class VersionedIntervalTimelineTest
ImmutableSet.of(
createExpected("2011-04-01/2011-04-09", "1", 1)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1346,7 +1357,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3),
createExpected("2011-04-09/2011-04-12", "1", 4)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1364,7 +1375,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3),
createExpected("2011-04-06/2011-04-09", "1", 4)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1379,7 +1390,7 @@ public class VersionedIntervalTimelineTest
assertValues(
ImmutableSet.of(),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1394,7 +1405,7 @@ public class VersionedIntervalTimelineTest
assertValues(
ImmutableSet.of(),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1409,7 +1420,7 @@ public class VersionedIntervalTimelineTest
assertValues(
ImmutableSet.of(),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1427,7 +1438,7 @@ public class VersionedIntervalTimelineTest
ImmutableSet.of(
createExpected("2011-04-03/2011-04-06", "1", 3)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1447,7 +1458,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-03/2011-04-06", "1", 3),
createExpected("2011-04-09/2011-04-12", "1", 3)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1465,7 +1476,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-01/2011-04-09", "2", 3),
createExpected("2011-04-01/2011-04-09", "1", 1)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1484,7 +1495,7 @@ public class VersionedIntervalTimelineTest
createExpected("2011-04-01/2011-04-09", "2", 3),
createExpected("2011-04-01/2011-04-09", "1", 1)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
}
@@ -1504,30 +1515,30 @@ public class VersionedIntervalTimelineTest
{
timeline = makeStringIntegerTimeline();
- add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(null, 1, 0, 77));
- add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(1, null, 1, 88));
- add("2011-04-01/2011-04-02", "2", IntegerPartitionChunk.make(null, 1, 0, 99));
+ add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("1", 0, 77)));
+ add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger("1", 1, 88)));
+ add("2011-04-01/2011-04-02", "2", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("2", 0, 99)));
assertValues(
ImmutableList.of(
createExpected("2011-04-01/2011-04-02", "1",
Arrays.asList(
- IntegerPartitionChunk.make(null, 1, 0, 77),
- IntegerPartitionChunk.make(1, null, 1, 88)
+ IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("1", 0, 77)),
+ IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger("1", 1, 88))
)
)
),
timeline.lookup(Intervals.of("2011-04-01/2011-04-02"))
);
- add("2011-04-01/2011-04-02", "3", IntegerPartitionChunk.make(null, 1, 0, 110));
+ add("2011-04-01/2011-04-02", "3", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("3", 0, 110)));
assertValues(
ImmutableList.of(
createExpected("2011-04-01/2011-04-02", "1",
Arrays.asList(
- IntegerPartitionChunk.make(null, 1, 0, 77),
- IntegerPartitionChunk.make(1, null, 1, 88)
+ IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("1", 0, 77)),
+ IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger("1", 1, 88))
)
)
),
@@ -1537,11 +1548,11 @@ public class VersionedIntervalTimelineTest
Sets.newHashSet(
createExpected("2011-04-01/2011-04-02", "2",
Collections.singletonList(
- IntegerPartitionChunk.make(null, 1, 0, 99)
+ IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("2", 0, 99))
)
)
),
- timeline.findOvershadowed()
+ timeline.findFullyOvershadowed()
);
testRemove();
@@ -1550,8 +1561,8 @@ public class VersionedIntervalTimelineTest
ImmutableList.of(
createExpected("2011-04-01/2011-04-02", "1",
Arrays.asList(
- IntegerPartitionChunk.make(null, 1, 0, 77),
- IntegerPartitionChunk.make(1, null, 1, 88)
+ IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("1", 0, 77)),
+ IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger("1", 1, 88))
)
)
),
@@ -1564,64 +1575,64 @@ public class VersionedIntervalTimelineTest
{
timeline = makeStringIntegerTimeline();
- add("2011-04-05/2011-04-07", "1", new SingleElementPartitionChunk<>(1));
- add("2011-04-07/2011-04-09", "1", new SingleElementPartitionChunk<>(1));
-
- add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk<>(1));
- add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk<>(1));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "1"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "1"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "1"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "1"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "2"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "2"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "2"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "2"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-30"), "0"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-08"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-09"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-10"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-30"), "0"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-09"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-10"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-30"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-10"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-15"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-17"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-19"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-30"), "0"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-16"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-17"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-18"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-19"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-20"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-30"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-19/2011-04-20"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0"));
+ add("2011-04-05/2011-04-07", "1", makeSingle("1", 1));
+ add("2011-04-07/2011-04-09", "1", makeSingle("1", 1));
+
+ add("2011-04-15/2011-04-17", "1", makeSingle("1", 1));
+ add("2011-04-17/2011-04-19", "1", makeSingle("1", 1));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "1", new OvershadowableInteger("1", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "1", new OvershadowableInteger("1", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "1", new OvershadowableInteger("1", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "1", new OvershadowableInteger("1", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "2", new OvershadowableInteger("2", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "2", new OvershadowableInteger("2", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "2", new OvershadowableInteger("2", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "2", new OvershadowableInteger("2", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-30"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-08"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-30"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-30"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-15"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-17"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-19"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-30"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-16"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-17"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-18"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-19"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-20"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-30"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-19/2011-04-20"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0", new OvershadowableInteger("0", 0, 1)));
}
@Test
@@ -1629,87 +1640,370 @@ public class VersionedIntervalTimelineTest
{
timeline = makeStringIntegerTimeline();
- add("2011-04-05/2011-04-09", "11", new SingleElementPartitionChunk<>(1));
- add("2011-04-07/2011-04-11", "12", new SingleElementPartitionChunk<>(1));
-
- add("2011-04-15/2011-04-19", "12", new SingleElementPartitionChunk<>(1));
- add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk<>(1));
-
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-11"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "12"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "12"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "12"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "12"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "12"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "12"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "13"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "13"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "13"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "13"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "13"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "13"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-12"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-15"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-16"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-17"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-18"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-19"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-20"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-21"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-22"), "0"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0"));
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-11"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-12"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-15"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-16"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-17"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-18"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-19"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-20"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-21"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-22"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-15"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-16"), "0"));
-
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-17"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-18"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-19"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-20"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-21"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-22"), "0"));
-
- Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-21"), "0"));
- Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0"));
- }
-
- private Pair<Interval, Pair<String, PartitionHolder<Integer>>> createExpected(
+ add("2011-04-05/2011-04-09", "11", makeSingle("11", 1));
+ add("2011-04-07/2011-04-11", "12", makeSingle("12", 1));
+
+ add("2011-04-15/2011-04-19", "12", makeSingle("12", 1));
+ add("2011-04-17/2011-04-21", "11", makeSingle("11", 1));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-11"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "12", new OvershadowableInteger("12", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "12", new OvershadowableInteger("12", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "12", new OvershadowableInteger("12", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "12", new OvershadowableInteger("12", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "12", new OvershadowableInteger("12", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "12", new OvershadowableInteger("12", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "13", new OvershadowableInteger("13", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "13", new OvershadowableInteger("13", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "13", new OvershadowableInteger("13", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "13", new OvershadowableInteger("13", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "13", new OvershadowableInteger("13", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "13", new OvershadowableInteger("13", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-12"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-15"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-16"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-17"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-18"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-19"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-20"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-21"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-22"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-11"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-12"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-15"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-16"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-17"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-18"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-19"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-20"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-21"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-22"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-15"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-16"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-17"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-18"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-19"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-20"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-21"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-22"), "0", new OvershadowableInteger("0", 0, 1)));
+
+ Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-21"), "0", new OvershadowableInteger("0", 0, 1)));
+ Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0", new OvershadowableInteger("0", 0, 1)));
+ }
+
+ @Test
+ public void testOvershadowedByReference()
+ {
+ timeline = makeStringIntegerTimeline();
+
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 2, 0));
+
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 0, 1, 0, 3, 1, 2));
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 1, 1, 0, 3, 1, 2));
+
+ Assert.assertEquals(
+ ImmutableSet.of(
+ makeTimelineObjectHolder(
+ "2019-01-01/2019-01-02",
+ "0",
+ ImmutableList.of(makeNumbered("0", 0, 0), makeNumbered("0", 1, 0), makeNumbered("0", 2, 0))
+ )
+ ),
+ timeline.findFullyOvershadowed()
+ );
+ }
+
+ @Test
+ public void testOvershadowedByReferenceChain()
+ {
+ timeline = makeStringIntegerTimeline();
+
+ // 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 2, 0));
+
+ // 2019-01-02/2019-01-03
+ add("2019-01-02/2019-01-03", "0", makeNumbered("0", 0, 0));
+ add("2019-01-02/2019-01-03", "0", makeNumbered("0", 1, 0));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 0, 1, 0, 3, 1, 2));
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 1, 1, 0, 3, 1, 2));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 2, 2, 0, 3, 2, 2));
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 3, 2, 0, 3, 2, 2));
+
+ Assert.assertEquals(
+ ImmutableSet.of(
+ makeTimelineObjectHolder(
+ "2019-01-01/2019-01-02",
+ "0",
+ ImmutableList.of(
+ makeNumbered("0", 0, 0),
+ makeNumbered("0", 1, 0),
+ makeNumbered("0", 2, 0),
+ makeNumberedOverwriting("0", 0, 1, 0, 3, 1, 2),
+ makeNumberedOverwriting("0", 1, 1, 0, 3, 1, 2)
+ )
+ )
+ ),
+ timeline.findFullyOvershadowed()
+ );
+ }
+
+ @Test
+ public void testOvershadowedByReferenceAndThenVersion()
+ {
+ timeline = makeStringIntegerTimeline();
+
+ // 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 2, 0));
+
+ // 2019-01-02/2019-01-03
+ add("2019-01-02/2019-01-03", "0", makeNumbered("0", 0, 0));
+ add("2019-01-02/2019-01-03", "0", makeNumbered("0", 1, 0));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 0, 1, 0, 3, 1, 2));
+ add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting("0", 1, 1, 0, 3, 1, 2));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "1", makeNumbered("1", 0, 0));
+ add("2019-01-01/2019-01-02", "1", makeNumbered("1", 1, 0));
+
+ Assert.assertEquals(
+ ImmutableSet.of(
+ makeTimelineObjectHolder(
+ "2019-01-01/2019-01-02",
+ "0",
+ ImmutableList.of(
+ makeNumbered("0", 0, 0),
+ makeNumbered("0", 1, 0),
+ makeNumbered("0", 2, 0),
+ makeNumberedOverwriting("0", 0, 1, 0, 3, 1, 2),
+ makeNumberedOverwriting("0", 1, 1, 0, 3, 1, 2)
+ )
+ )
+ ),
+ timeline.findFullyOvershadowed()
+ );
+ }
+
+ @Test
+ public void testOvershadowedByVersionAndThenReference()
+ {
+ timeline = makeStringIntegerTimeline();
+
+ // 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0));
+ add("2019-01-01/2019-01-02", "0", makeNumbered("0", 2, 0));
+
+ // 2019-01-02/2019-01-03
+ add("2019-01-02/2019-01-03", "0", makeNumbered("0", 0, 0));
+ add("2019-01-02/2019-01-03", "0", makeNumbered("0", 1, 0));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "1", makeNumbered("1", 0, 0));
+ add("2019-01-01/2019-01-02", "1", makeNumbered("1", 1, 0));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting("1", 0, 1, 0, 2, 1, 3));
+ add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting("1", 1, 1, 0, 2, 1, 3));
+ add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting("1", 2, 1, 0, 2, 1, 3));
+
+ Assert.assertEquals(
+ ImmutableSet.of(
+ makeTimelineObjectHolder(
+ "2019-01-01/2019-01-02",
+ "0",
+ ImmutableList.of(
+ makeNumbered("0", 0, 0),
+ makeNumbered("0", 1, 0),
+ makeNumbered("0", 2, 0)
+ )
+ ),
+ makeTimelineObjectHolder(
+ "2019-01-01/2019-01-02",
+ "1",
+ ImmutableList.of(
+ makeNumbered("1", 0, 0),
+ makeNumbered("1", 1, 0)
+ )
+ )
+ ),
+ timeline.findFullyOvershadowed()
+ );
+ }
+
+ @Test
+ public void testFallbackOnMissingSegment()
+ {
+ timeline = makeStringIntegerTimeline();
+
+ final Interval interval = Intervals.of("2019-01-01/2019-01-02");
+
+ add(interval, "0", makeNumbered("0", 0, 0));
+ add(interval, "0", makeNumbered("0", 1, 0));
+ add(interval, "0", makeNumbered("0", 2, 0));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add(interval, "1", makeNumbered("1", 0, 0));
+ add(interval, "1", makeNumbered("1", 1, 0));
+
+ // Overwrite 2019-01-01/2019-01-02
+ add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting("1", 0, 1, 0, 2, 1, 3));
+ add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting("1", 1, 1, 0, 2, 1, 3));
+ add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting("1", 2, 1, 0, 2, 1, 3));
+
+ timeline.remove(
+ interval,
+ "1",
+ makeNumberedOverwriting("1", 2, 1, 0, 2, 1, 3)
+ );
+
+ final List<TimelineObjectHolder<String, OvershadowableInteger>> holders = timeline.lookup(interval);
+
+ Assert.assertEquals(
+ ImmutableList.of(
+ new TimelineObjectHolder<>(
+ interval,
+ "1",
+ new PartitionHolder<>(
+ ImmutableList.of(
+ makeNumbered("1", 0, 0),
+ makeNumbered("1", 1, 0),
+ makeNumberedOverwriting("1", 0, 1, 0, 2, 1, 3),
+ makeNumberedOverwriting("1", 1, 1, 0, 2, 1, 3)
+ )
+ )
+ )
+ ),
+ holders
+ );
+ }
+
+ @Test
+ public void testAddSameChunkToFullAtomicUpdateGroup()
+ {
+ timeline = makeStringIntegerTimeline();
+ final Interval interval = Intervals.of("2019-01-01/2019-01-02");
+ add(interval, "0", makeNumbered("0", 0, 0));
+ add(interval, "0", makeNumberedOverwriting("0", 0, 0, 0, 1, 1, 1));
+ add(interval, "0", makeNumbered("0", 0, 1));
+
+ final Set<TimelineObjectHolder<String, OvershadowableInteger>> overshadowed = timeline.findFullyOvershadowed();
+ Assert.assertEquals(
+ ImmutableSet.of(
+ new TimelineObjectHolder<>(
+ interval,
+ "0",
+ new PartitionHolder<>(ImmutableList.of(makeNumbered("0", 0, 1)))
+ )
+ ),
+ overshadowed
+ );
+ }
+
+ @Test
+ public void testOvershadowMultipleStandbyAtomicUpdateGroup()
+ {
+ timeline = makeStringIntegerTimeline();
+ final Interval interval = Intervals.of("2019-01-01/2019-01-02");
+ add(interval, "0", makeNumberedOverwriting("0", 0, 0, 0, 1, 1, 2));
+ add(interval, "0", makeNumberedOverwriting("0", 1, 0, 0, 1, 2, 2));
+ add(interval, "0", makeNumberedOverwriting("0", 2, 0, 0, 1, 3, 2)); // <-- full atomicUpdateGroup
+ add(interval, "0", makeNumberedOverwriting("0", 3, 1, 0, 1, 3, 2)); // <-- full atomicUpdateGroup
+
+ final Set<TimelineObjectHolder<String, OvershadowableInteger>> overshadowed = timeline.findFullyOvershadowed();
+ Assert.assertEquals(
+ ImmutableSet.of(
+ new TimelineObjectHolder<>(
+ interval,
+ "0",
+ new PartitionHolder<>(
+ ImmutableList.of(
+ makeNumberedOverwriting("0", 0, 0, 0, 1, 1, 2),
+ makeNumberedOverwriting("0", 1, 0, 0, 1, 2, 2)
+ )
+ )
+ )
+ ),
+ overshadowed
+ );
+ }
+
+ @Test
+ public void testIsOvershadowedForOverwritingSegments()
+ {
+ timeline = makeStringIntegerTimeline();
+ final Interval interval = Intervals.of("2019-01-01/2019-01-02");
+ add(interval, "0", makeNumberedOverwriting("0", 0, 0, 5, 10, 10, 1));
+
+ for (int i = 0; i < 5; i++) {
+ Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumbered("0", i + 5, 0).getObject()));
+ }
+
+ Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumbered("0", 4, 0).getObject()));
+ Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumbered("0", 11, 0).getObject()));
+
+ Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting("0", 1, 0, 5, 6, 5, 2).getObject()));
+ Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting("0", 1, 0, 7, 8, 5, 2).getObject()));
+ Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting("0", 1, 0, 8, 10, 5, 2).getObject()));
+
+ Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting("0", 1, 0, 5, 10, 12, 2).getObject()));
+ Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting("0", 1, 0, 4, 15, 12, 2).getObject()));
+ }
+
+ private TimelineObjectHolder<String, OvershadowableInteger> makeTimelineObjectHolder(
+ String interval,
+ String version,
+ List<PartitionChunk<OvershadowableInteger>> chunks
+ )
+ {
+ return new TimelineObjectHolder<>(
+ Intervals.of(interval),
+ Intervals.of(interval),
+ version,
+ new PartitionHolder<>(chunks)
+ );
+ }
+
+ private Pair<Interval, Pair<String, PartitionHolder<OvershadowableInteger>>> createExpected(
String intervalString,
String version,
Integer value
@@ -1718,14 +2012,14 @@ public class VersionedIntervalTimelineTest
return createExpected(
intervalString,
version,
- Collections.singletonList(makeSingle(value))
+ Collections.singletonList(makeSingle(version, value))
);
}
- private Pair<Interval, Pair<String, PartitionHolder<Integer>>> createExpected(
+ private Pair<Interval, Pair<String, PartitionHolder<OvershadowableInteger>>> createExpected(
String intervalString,
String version,
- List<PartitionChunk<Integer>> values
+ List<PartitionChunk<OvershadowableInteger>> values
)
{
return Pair.of(
@@ -1734,9 +2028,48 @@ public class VersionedIntervalTimelineTest
);
}
- private SingleElementPartitionChunk<Integer> makeSingle(Integer value)
+ private PartitionChunk<OvershadowableInteger> makeSingle(String majorVersion, int value)
+ {
+ return makeSingle(majorVersion, 0, value);
+ }
+
+ private PartitionChunk<OvershadowableInteger> makeSingle(String majorVersion, int partitionNum, int val)
{
- return new SingleElementPartitionChunk<>(value);
+ return new SingleElementPartitionChunk<>(new OvershadowableInteger(majorVersion, partitionNum, val));
+ }
+
+ private PartitionChunk<OvershadowableInteger> makeNumbered(String majorVersion, int partitionNum, int val)
+ {
+ return new NumberedPartitionChunk<>(
+ partitionNum,
+ 0,
+ new OvershadowableInteger(majorVersion, partitionNum, val)
+ );
+ }
+
+ private PartitionChunk<OvershadowableInteger> makeNumberedOverwriting(
+ String majorVersion,
+ int partitionNumOrdinal,
+ int val,
+ int startRootPartitionId,
+ int endRootPartitionId,
+ int minorVersion,
+ int atomicUpdateGroupSize
+ )
+ {
+ final int partitionNum = PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + partitionNumOrdinal;
+ return new NumberedOverwritingPartitionChunk<>(
+ partitionNum,
+ new OvershadowableInteger(
+ majorVersion,
+ partitionNum,
+ val,
+ startRootPartitionId,
+ endRootPartitionId,
+ minorVersion,
+ atomicUpdateGroupSize
+ )
+ );
}
private void add(String interval, String version, Integer value)
@@ -1746,69 +2079,182 @@ public class VersionedIntervalTimelineTest
private void add(Interval interval, String version, Integer value)
{
- add(interval, version, makeSingle(value));
+ add(interval, version, makeSingle(version, value));
}
- private void add(String interval, String version, PartitionChunk<Integer> value)
+ private void add(String interval, String version, PartitionChunk<OvershadowableInteger> value)
{
add(Intervals.of(interval), version, value);
}
- private void add(Interval interval, String version, PartitionChunk<Integer> value)
+ private void add(Interval interval, String version, PartitionChunk<OvershadowableInteger> value)
{
timeline.add(interval, version, value);
}
private void assertValues(
- List<Pair<Interval, Pair<String, PartitionHolder<Integer>>>> expected,
- List<TimelineObjectHolder<String, Integer>> actual
+ List<Pair<Interval, Pair<String, PartitionHolder<OvershadowableInteger>>>> expected,
+ List<TimelineObjectHolder<String, OvershadowableInteger>> actual
)
{
Assert.assertEquals("Sizes did not match.", expected.size(), actual.size());
- Iterator<Pair<Interval, Pair<String, PartitionHolder<Integer>>>> expectedIter = expected.iterator();
- Iterator<TimelineObjectHolder<String, Integer>> actualIter = actual.iterator();
+ Iterator<Pair<Interval, Pair<String, PartitionHolder<OvershadowableInteger>>>> expectedIter = expected.iterator();
+ Iterator<TimelineObjectHolder<String, OvershadowableInteger>> actualIter = actual.iterator();
while (expectedIter.hasNext()) {
- Pair<Interval, Pair<String, PartitionHolder<Integer>>> pair = expectedIter.next();
- TimelineObjectHolder<String, Integer> holder = actualIter.next();
+ Pair<Interval, Pair<String, PartitionHolder<OvershadowableInteger>>> pair = expectedIter.next();
+ TimelineObjectHolder<String, OvershadowableInteger> holder = actualIter.next();
Assert.assertEquals(pair.lhs, holder.getInterval());
Assert.assertEquals(pair.rhs.lhs, holder.getVersion());
- Assert.assertEquals(pair.rhs.rhs, holder.getObject());
+
+ final List<PartitionChunk<OvershadowableInteger>> expectedChunks = Lists.newArrayList(pair.rhs.rhs);
+ final List<PartitionChunk<OvershadowableInteger>> actualChunks = Lists.newArrayList(holder.getObject());
+
+ Assert.assertEquals(expectedChunks.size(), actualChunks.size());
+ for (int i = 0; i < expectedChunks.size(); i++) {
+ // Check partitionNumber first
+ Assert.assertEquals(expectedChunks.get(i), actualChunks.get(i));
+ final OvershadowableInteger expectedInteger = expectedChunks.get(i).getObject();
+ final OvershadowableInteger actualInteger = actualChunks.get(i).getObject();
+ Assert.assertEquals(expectedInteger, actualInteger);
+ }
}
}
private void assertValues(
- Set<Pair<Interval, Pair<String, PartitionHolder<Integer>>>> expected,
- Set<TimelineObjectHolder<String, Integer>> actual
+ Set<Pair<Interval, Pair<String, PartitionHolder<OvershadowableInteger>>>> expected,
+ Set<TimelineObjectHolder<String, OvershadowableInteger>> actual
)
{
Assert.assertEquals("Sizes did not match.", expected.size(), actual.size());
- Set<Pair<Interval, Pair<String, PartitionHolder<Integer>>>> actualSet =
+ Set<Pair<Interval, Pair<String, PartitionHolder<OvershadowableInteger>>>> actualSet =
Sets.newHashSet(
Iterables.transform(
actual,
- new Function<TimelineObjectHolder<String, Integer>, Pair<Interval, Pair<String, PartitionHolder<Integer>>>>()
- {
- @Override
- public Pair<Interval, Pair<String, PartitionHolder<Integer>>> apply(
- TimelineObjectHolder<String, Integer> input
- )
- {
- return new Pair<>(input.getInterval(), new Pair<>(input.getVersion(), input.getObject()));
- }
- }
+ input -> new Pair<>(input.getInterval(), new Pair<>(input.getVersion(), input.getObject()))
)
);
Assert.assertEquals(expected, actualSet);
}
- private VersionedIntervalTimeline<String, Integer> makeStringIntegerTimeline()
+ private VersionedIntervalTimeline<String, OvershadowableInteger> makeStringIntegerTimeline()
{
return new VersionedIntervalTimeline<>(Ordering.natural());
}
+ private static class OvershadowableInteger implements Overshadowable<OvershadowableInteger>
+ {
+ private final String majorVersion;
+ private final int partitionNum;
+ private final int val;
+ private final int startRootPartitionId;
+ private final int endRootPartitionId;
+ private final short minorVersion;
+ private final short atomicUpdateGroupSize;
+
+ private OvershadowableInteger(String majorVersion, int partitionNum, int val)
+ {
+ this(majorVersion, partitionNum, val, partitionNum, partitionNum + 1, 0, 1);
+ }
+
+ private OvershadowableInteger(
+ String majorVersion,
+ int partitionNum,
+ int val,
+ int startRootPartitionId,
+ int endRootPartitionId,
+ int minorVersion,
+ int atomicUpdateGroupSize
+ )
+ {
+ this.majorVersion = majorVersion;
+ this.partitionNum = partitionNum;
+ this.val = val;
+ this.startRootPartitionId = startRootPartitionId;
+ this.endRootPartitionId = endRootPartitionId;
+ this.minorVersion = (short) minorVersion;
+ this.atomicUpdateGroupSize = (short) atomicUpdateGroupSize;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ OvershadowableInteger that = (OvershadowableInteger) o;
+ return partitionNum == that.partitionNum &&
+ val == that.val &&
+ startRootPartitionId == that.startRootPartitionId &&
+ endRootPartitionId == that.endRootPartitionId &&
+ minorVersion == that.minorVersion &&
+ atomicUpdateGroupSize == that.atomicUpdateGroupSize &&
+ Objects.equals(majorVersion, that.majorVersion);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(
+ majorVersion,
+ partitionNum,
+ val,
+ startRootPartitionId,
+ endRootPartitionId,
+ minorVersion,
+ atomicUpdateGroupSize
+ );
+ }
+
+ @Override
+ public String toString()
+ {
+ return "OvershadowableInteger{" +
+ "majorVersion='" + majorVersion + '\'' +
+ ", partitionNum=" + partitionNum +
+ ", val=" + val +
+ ", startRootPartitionId=" + startRootPartitionId +
+ ", endRootPartitionId=" + endRootPartitionId +
+ ", minorVersion=" + minorVersion +
+ ", atomicUpdateGroupSize=" + atomicUpdateGroupSize +
+ '}';
+ }
+
+ @Override
+ public int getStartRootPartitionId()
+ {
+ return startRootPartitionId;
+ }
+
+ @Override
+ public int getEndRootPartitionId()
+ {
+ return endRootPartitionId;
+ }
+
+ @Override
+ public String getVersion()
+ {
+ return majorVersion;
+ }
+
+ @Override
+ public short getMinorVersion()
+ {
+ return minorVersion;
+ }
+
+ @Override
+ public short getAtomicUpdateGroupSize()
+ {
+ return atomicUpdateGroupSize;
+ }
+ }
}
diff --git a/core/src/test/java/org/apache/druid/timeline/partition/IntegerPartitionChunkTest.java b/core/src/test/java/org/apache/druid/timeline/partition/IntegerPartitionChunkTest.java
index 36b3915..04e4ccf 100644
--- a/core/src/test/java/org/apache/druid/timeline/partition/IntegerPartitionChunkTest.java
+++ b/core/src/test/java/org/apache/druid/timeline/partition/IntegerPartitionChunkTest.java
@@ -19,86 +19,135 @@
package org.apache.druid.timeline.partition;
+import org.apache.druid.timeline.Overshadowable;
import org.junit.Assert;
import org.junit.Test;
public class IntegerPartitionChunkTest
{
+ private static IntegerPartitionChunk<OvershadowableInteger> make(
+ Integer start,
+ Integer end,
+ int chunkNumber,
+ int obj
+ )
+ {
+ return new IntegerPartitionChunk<>(start, end, chunkNumber, new OvershadowableInteger(obj));
+ }
+
@Test
public void testAbuts()
{
- IntegerPartitionChunk<Integer> lhs = IntegerPartitionChunk.make(null, 10, 0, 1);
+ IntegerPartitionChunk<OvershadowableInteger> lhs = make(null, 10, 0, 1);
- Assert.assertTrue(lhs.abuts(IntegerPartitionChunk.make(10, null, 1, 2)));
- Assert.assertFalse(lhs.abuts(IntegerPartitionChunk.make(11, null, 2, 3)));
- Assert.assertFalse(lhs.abuts(IntegerPartitionChunk.make(null, null, 3, 4)));
+ Assert.assertTrue(lhs.abuts(make(10, null, 1, 2)));
+ Assert.assertFalse(lhs.abuts(make(11, null, 2, 3)));
+ Assert.assertFalse(lhs.abuts(make(null, null, 3, 4)));
- Assert.assertFalse(IntegerPartitionChunk.make(null, null, 0, 1)
- .abuts(IntegerPartitionChunk.make(null, null, 1, 2)));
+ Assert.assertFalse(make(null, null, 0, 1).abuts(make(null, null, 1, 2)));
}
@Test
public void testIsStart()
{
- Assert.assertTrue(IntegerPartitionChunk.make(null, 10, 0, 1).isStart());
- Assert.assertFalse(IntegerPartitionChunk.make(10, null, 0, 1).isStart());
- Assert.assertFalse(IntegerPartitionChunk.make(10, 11, 0, 1).isStart());
- Assert.assertTrue(IntegerPartitionChunk.make(null, null, 0, 1).isStart());
+ Assert.assertTrue(make(null, 10, 0, 1).isStart());
+ Assert.assertFalse(make(10, null, 0, 1).isStart());
+ Assert.assertFalse(make(10, 11, 0, 1).isStart());
+ Assert.assertTrue(make(null, null, 0, 1).isStart());
}
@Test
public void testIsEnd()
{
- Assert.assertFalse(IntegerPartitionChunk.make(null, 10, 0, 1).isEnd());
- Assert.assertTrue(IntegerPartitionChunk.make(10, null, 0, 1).isEnd());
- Assert.assertFalse(IntegerPartitionChunk.make(10, 11, 0, 1).isEnd());
- Assert.assertTrue(IntegerPartitionChunk.make(null, null, 0, 1).isEnd());
+ Assert.assertFalse(make(null, 10, 0, 1).isEnd());
+ Assert.assertTrue(make(10, null, 0, 1).isEnd());
+ Assert.assertFalse(make(10, 11, 0, 1).isEnd());
+ Assert.assertTrue(make(null, null, 0, 1).isEnd());
}
@Test
public void testCompareTo()
{
- //noinspection EqualsWithItself (the intention of this first test is specifically to call compareTo with itself)
Assert.assertEquals(
0,
- IntegerPartitionChunk.make(null, null, 0, 1).compareTo(IntegerPartitionChunk.make(null, null, 0, 1))
+ make(null, null, 0, 1).compareTo(make(null, null, 0, 1))
);
Assert.assertEquals(
0,
- IntegerPartitionChunk.make(10, null, 0, 1).compareTo(IntegerPartitionChunk.make(10, null, 0, 2))
+ make(10, null, 0, 1).compareTo(make(10, null, 0, 2))
);
Assert.assertEquals(
0,
- IntegerPartitionChunk.make(null, 10, 0, 1).compareTo(IntegerPartitionChunk.make(null, 10, 0, 2))
+ make(null, 10, 0, 1).compareTo(make(null, 10, 0, 2))
);
Assert.assertEquals(
0,
- IntegerPartitionChunk.make(10, 11, 0, 1).compareTo(IntegerPartitionChunk.make(10, 11, 0, 2))
+ make(10, 11, 0, 1).compareTo(make(10, 11, 0, 2))
);
Assert.assertEquals(
-1,
- IntegerPartitionChunk.make(null, 10, 0, 1).compareTo(IntegerPartitionChunk.make(10, null, 1, 2))
+ make(null, 10, 0, 1).compareTo(make(10, null, 1, 2))
);
Assert.assertEquals(
-1,
- IntegerPartitionChunk.make(11, 20, 0, 1).compareTo(IntegerPartitionChunk.make(20, 33, 1, 1))
+ make(11, 20, 0, 1).compareTo(make(20, 33, 1, 1))
);
Assert.assertEquals(
1,
- IntegerPartitionChunk.make(20, 33, 1, 1).compareTo(IntegerPartitionChunk.make(11, 20, 0, 1))
+ make(20, 33, 1, 1).compareTo(make(11, 20, 0, 1))
);
Assert.assertEquals(
1,
- IntegerPartitionChunk.make(10, null, 1, 1).compareTo(IntegerPartitionChunk.make(null, 10, 0, 1))
+ make(10, null, 1, 1).compareTo(make(null, 10, 0, 1))
);
}
@Test
public void testEquals()
{
- Assert.assertEquals(IntegerPartitionChunk.make(null, null, 0, 1), IntegerPartitionChunk.make(null, null, 0, 1));
- Assert.assertEquals(IntegerPartitionChunk.make(null, 10, 0, 1), IntegerPartitionChunk.make(null, 10, 0, 1));
- Assert.assertEquals(IntegerPartitionChunk.make(10, null, 0, 1), IntegerPartitionChunk.make(10, null, 0, 1));
- Assert.assertEquals(IntegerPartitionChunk.make(10, 11, 0, 1), IntegerPartitionChunk.make(10, 11, 0, 1));
+ Assert.assertEquals(make(null, null, 0, 1), make(null, null, 0, 1));
+ Assert.assertEquals(make(null, 10, 0, 1), make(null, 10, 0, 1));
+ Assert.assertEquals(make(10, null, 0, 1), make(10, null, 0, 1));
+ Assert.assertEquals(make(10, 11, 0, 1), make(10, 11, 0, 1));
+ }
+
+ private static class OvershadowableInteger implements Overshadowable<OvershadowableInteger>
+ {
+ private final int val;
+
+ OvershadowableInteger(int val)
+ {
+ this.val = val;
+ }
+
+ @Override
+ public int getStartRootPartitionId()
+ {
+ return 0;
+ }
+
+ @Override
+ public int getEndRootPartitionId()
+ {
+ return 1;
+ }
+
+ @Override
+ public String getVersion()
+ {
+ return "";
+ }
+
+ @Override
+ public short getMinorVersion()
+ {
+ return 0;
+ }
+
+ @Override
+ public short getAtomicUpdateGroupSize()
+ {
+ return 1;
+ }
}
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java
index 3b34454..c77725d 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import org.apache.druid.data.input.impl.InputRowParser;
+import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
@@ -70,7 +71,8 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
AuthorizerMapper authorizerMapper,
Optional<ChatHandlerProvider> chatHandlerProvider,
CircularBuffer<Throwable> savedParseExceptions,
- RowIngestionMetersFactory rowIngestionMetersFactory
+ RowIngestionMetersFactory rowIngestionMetersFactory,
+ LockGranularity lockGranularityToUse
)
{
super(
@@ -79,7 +81,8 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd
authorizerMapper,
chatHandlerProvider,
savedParseExceptions,
- rowIngestionMetersFactory
+ rowIngestionMetersFactory,
+ lockGranularityToUse
);
this.task = task;
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
index b0b0c6b..25140d8 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
@@ -135,7 +135,8 @@ public class KafkaIndexTask extends SeekableStreamIndexTask<Integer, Long>
authorizerMapper,
chatHandlerProvider,
savedParseExceptions,
- rowIngestionMetersFactory
+ rowIngestionMetersFactory,
+ lockGranularityToUse
);
}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
index d906029..87f522e 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -24,7 +24,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Predicates;
-import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@@ -50,6 +49,7 @@ import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
+import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskReport;
@@ -67,6 +67,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMeters;
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.common.task.IndexTaskTest;
import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.kafka.test.TestBroker;
@@ -89,6 +90,7 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.concurrent.ListenableFutures;
import org.apache.druid.java.util.common.granularity.Granularities;
+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.parsers.JSONPathSpec;
import org.apache.druid.java.util.emitter.EmittingLogger;
@@ -162,6 +164,8 @@ import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.File;
@@ -171,6 +175,7 @@ import java.lang.reflect.Method;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -183,7 +188,9 @@ import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+@RunWith(Parameterized.class)
public class KafkaIndexTaskTest
{
private static final Logger log = new Logger(KafkaIndexTaskTest.class);
@@ -200,7 +207,17 @@ public class KafkaIndexTaskTest
new KafkaIndexTaskModule().getJacksonModules().forEach(OBJECT_MAPPER::registerModule);
}
+ @Parameterized.Parameters(name = "{0}")
+ public static Iterable<Object[]> constructorFeeder()
+ {
+ return ImmutableList.of(
+ new Object[]{LockGranularity.TIME_CHUNK},
+ new Object[]{LockGranularity.SEGMENT}
+ );
+ }
+
private final List<Task> runningTasks = new ArrayList<>();
+ private final LockGranularity lockGranularity;
private long handoffConditionTimeout = 0;
private boolean reportParseExceptions = false;
@@ -309,6 +326,11 @@ public class KafkaIndexTaskTest
@Rule
public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule();
+ public KafkaIndexTaskTest(LockGranularity lockGranularity)
+ {
+ this.lockGranularity = lockGranularity;
+ }
+
@BeforeClass
public static void setupClass() throws Exception
{
@@ -414,17 +436,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -464,17 +487,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -553,14 +577,14 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
- SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
- SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2010/P1D", 0);
+ SegmentDescriptor desc4 = sd("2011/P1D", 0);
+ SegmentDescriptor desc5 = sd("2011/P1D", 1);
+ SegmentDescriptor desc6 = sd("2012/P1D", 0);
+ SegmentDescriptor desc7 = sd("2013/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
@@ -569,15 +593,16 @@ public class KafkaIndexTaskTest
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
- Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4))
- && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) ||
- (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4))
- && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5))));
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
+ Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", publishedDescriptors.get(3)))
+ && ImmutableList.of("h").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))) ||
+ (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", publishedDescriptors.get(3)))
+ && ImmutableList.of("e").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))));
+ Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(5)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(6)));
}
@Test(timeout = 60_000L)
@@ -697,14 +722,14 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
- SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
- SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2010/P1D", 0);
+ SegmentDescriptor desc4 = sd("2011/P1D", 0);
+ SegmentDescriptor desc5 = sd("2011/P1D", 1);
+ SegmentDescriptor desc6 = sd("2012/P1D", 0);
+ SegmentDescriptor desc7 = sd("2013/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
@@ -712,7 +737,7 @@ public class KafkaIndexTaskTest
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
@@ -721,15 +746,16 @@ public class KafkaIndexTaskTest
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
- Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4))
- && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) ||
- (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4))
- && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5))));
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
+ Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", publishedDescriptors.get(3)))
+ && ImmutableList.of("h").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))) ||
+ (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", publishedDescriptors.get(3)))
+ && ImmutableList.of("e").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))));
+ Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(5)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(6)));
}
@Test(timeout = 60_000L)
@@ -804,9 +830,9 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L))
@@ -815,8 +841,9 @@ public class KafkaIndexTaskTest
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -957,17 +984,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -1007,19 +1035,20 @@ public class KafkaIndexTaskTest
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2010/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
}
@Test(timeout = 60_000L)
@@ -1067,16 +1096,17 @@ public class KafkaIndexTaskTest
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2009/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2009/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", publishedDescriptors.get(0)));
}
@Test(timeout = 60_000L)
@@ -1111,7 +1141,7 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+ Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
}
@Test(timeout = 60_000L)
@@ -1148,17 +1178,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -1196,9 +1227,10 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))
@@ -1207,8 +1239,8 @@ public class KafkaIndexTaskTest
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -1249,7 +1281,7 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+ Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
}
@@ -1293,11 +1325,11 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ SegmentDescriptor desc3 = sd("2013/P1D", 0);
+ SegmentDescriptor desc4 = sd("2049/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
@@ -1371,7 +1403,7 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+ Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
@@ -1449,17 +1481,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -1514,17 +1547,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata, should all be from the first task
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -1567,9 +1601,9 @@ public class KafkaIndexTaskTest
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
// Run second task
@@ -1585,16 +1619,17 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
- SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+ SegmentDescriptor desc3 = sd("2011/P1D", 1);
+ SegmentDescriptor desc4 = sd("2013/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(3)));
}
@Test(timeout = 60_000L)
@@ -1629,12 +1664,13 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
// desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments
- SegmentDescriptor desc3 = sd(task, "2011/P1D", 1);
- SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors());
+ SegmentDescriptor desc3 = sd("2011/P1D", 1);
+ SegmentDescriptor desc4 = sd("2012/P1D", 0);
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc4), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L))
@@ -1643,13 +1679,19 @@ public class KafkaIndexTaskTest
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(
+ ImmutableList.of("g"),
+ readSegmentColumn(
+ "dim1",
+ publishedDescriptors.get(2)
+ )
+ );
// Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically
Assert.assertEquals(
ImmutableSet.of(ImmutableList.of("d", "e", "h")),
- ImmutableSet.of(readSegmentColumn("dim1", desc2))
+ ImmutableSet.of(readSegmentColumn("dim1", publishedDescriptors.get(1)))
);
}
@@ -1704,10 +1746,10 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- SegmentDescriptor desc3 = sd(task2, "2012/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ SegmentDescriptor desc3 = sd("2012/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L))
@@ -1716,9 +1758,10 @@ public class KafkaIndexTaskTest
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc3));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
}
@Test(timeout = 60_000L)
@@ -1804,17 +1847,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -1913,14 +1957,14 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2008/P1D", 1);
- SegmentDescriptor desc3 = sd(task1, "2009/P1D", 0);
- SegmentDescriptor desc4 = sd(task1, "2009/P1D", 1);
- SegmentDescriptor desc5 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc6 = sd(task1, "2011/P1D", 0);
- SegmentDescriptor desc7 = sd(task1, "2012/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2008/P1D", 1);
+ SegmentDescriptor desc3 = sd("2009/P1D", 0);
+ SegmentDescriptor desc4 = sd("2009/P1D", 1);
+ SegmentDescriptor desc5 = sd("2010/P1D", 0);
+ SegmentDescriptor desc6 = sd("2011/P1D", 0);
+ SegmentDescriptor desc7 = sd("2012/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
@@ -2003,17 +2047,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -2126,17 +2171,18 @@ public class KafkaIndexTaskTest
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -2267,21 +2313,22 @@ public class KafkaIndexTaskTest
Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2013/P1D", 0);
+ SegmentDescriptor desc4 = sd("2049/P1D", 0);
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc3));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4));
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(3)));
}
@Test(timeout = 60_000L)
@@ -2391,6 +2438,7 @@ public class KafkaIndexTaskTest
return taskExec.submit(
() -> {
try {
+ task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK);
if (task.isReady(toolbox.getTaskActionClient())) {
return task.run(toolbox);
} else {
@@ -2587,7 +2635,7 @@ public class KafkaIndexTaskTest
derby.metadataTablesConfigSupplier().get(),
derbyConnector
);
- taskLockbox = new TaskLockbox(taskStorage);
+ taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator);
final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
taskLockbox,
taskStorage,
@@ -2690,14 +2738,12 @@ public class KafkaIndexTaskTest
metadataStorageCoordinator = null;
}
- private Set<SegmentDescriptor> publishedDescriptors()
+ private List<SegmentDescriptor> publishedDescriptors()
{
- return FluentIterable.from(
- metadataStorageCoordinator.getUsedSegmentsForInterval(
- DATA_SCHEMA.getDataSource(),
- Intervals.of("0000/3000")
- )
- ).transform(DataSegment::toDescriptor).toSet();
+ return metadataStorageCoordinator.getUsedSegmentsForInterval(
+ DATA_SCHEMA.getDataSource(),
+ Intervals.of("0000/3000")
+ ).stream().map(DataSegment::toDescriptor).collect(Collectors.toList());
}
private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task)
@@ -2796,10 +2842,33 @@ public class KafkaIndexTaskTest
}
}
- private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum)
+ private SegmentDescriptor sd(final String intervalString, final int partitionNum)
{
final Interval interval = Intervals.of(intervalString);
- return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
+ return new SegmentDescriptor(interval, "fakeVersion", partitionNum);
+ }
+
+ private void assertEqualsExceptVersion(List<SegmentDescriptor> descriptors1, List<SegmentDescriptor> descriptors2)
+ {
+ Assert.assertEquals(descriptors1.size(), descriptors2.size());
+ final Comparator<SegmentDescriptor> comparator = (s1, s2) -> {
+ final int intervalCompare = Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval());
+ if (intervalCompare == 0) {
+ return Integer.compare(s1.getPartitionNumber(), s2.getPartitionNumber());
+ } else {
+ return intervalCompare;
+ }
+ };
+
+ final List<SegmentDescriptor> copy1 = new ArrayList<>(descriptors1);
+ final List<SegmentDescriptor> copy2 = new ArrayList<>(descriptors2);
+ copy1.sort(comparator);
+ copy2.sort(comparator);
+
+ for (int i = 0; i < copy1.size(); i++) {
+ Assert.assertEquals(copy1.get(i).getInterval(), copy2.get(i).getInterval());
+ Assert.assertEquals(copy1.get(i).getPartitionNumber(), copy2.get(i).getPartitionNumber());
+ }
}
private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java
index f3dfe3b..3490ee9 100644
--- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java
@@ -68,7 +68,6 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
this.awsCredentialsConfig = awsCredentialsConfig;
}
-
@Override
protected SeekableStreamIndexTaskRunner<String, String> createTaskRunner()
{
@@ -79,7 +78,8 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String>
authorizerMapper,
chatHandlerProvider,
savedParseExceptions,
- rowIngestionMetersFactory
+ rowIngestionMetersFactory,
+ lockGranularityToUse
);
}
diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java
index 335119a..3e88bfd 100644
--- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java
+++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import org.apache.druid.data.input.impl.InputRowParser;
+import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
@@ -64,7 +65,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
AuthorizerMapper authorizerMapper,
Optional<ChatHandlerProvider> chatHandlerProvider,
CircularBuffer<Throwable> savedParseExceptions,
- RowIngestionMetersFactory rowIngestionMetersFactory
+ RowIngestionMetersFactory rowIngestionMetersFactory,
+ LockGranularity lockGranularityToUse
)
{
super(
@@ -73,7 +75,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
authorizerMapper,
chatHandlerProvider,
savedParseExceptions,
- rowIngestionMetersFactory
+ rowIngestionMetersFactory,
+ lockGranularityToUse
);
this.task = task;
}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
index 8eb4c14..006677b 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
@@ -30,7 +30,6 @@ import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Predicates;
import com.google.common.base.Throwables;
-import com.google.common.collect.FluentIterable;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@@ -56,6 +55,7 @@ import org.apache.druid.discovery.LookupNodeService;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
+import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.SegmentLoaderFactory;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskReport;
@@ -74,6 +74,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory;
import org.apache.druid.indexing.common.task.IndexTaskTest;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.TaskResource;
+import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
@@ -97,6 +98,7 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.concurrent.ListenableFutures;
import org.apache.druid.java.util.common.granularity.Granularities;
+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.parsers.JSONPathSpec;
import org.apache.druid.java.util.emitter.EmittingLogger;
@@ -162,6 +164,8 @@ import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.File;
@@ -172,6 +176,7 @@ import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -185,8 +190,10 @@ import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+@RunWith(Parameterized.class)
public class KinesisIndexTaskTest extends EasyMockSupport
{
private static final Logger LOG = new Logger(KinesisIndexTaskTest.class);
@@ -202,6 +209,17 @@ public class KinesisIndexTaskTest extends EasyMockSupport
private final List<Task> runningTasks = new ArrayList<>();
+ @Parameterized.Parameters(name = "{0}")
+ public static Iterable<Object[]> constructorFeeder()
+ {
+ return ImmutableList.of(
+ new Object[]{LockGranularity.TIME_CHUNK},
+ new Object[]{LockGranularity.SEGMENT}
+ );
+ }
+
+ private final LockGranularity lockGranularity;
+
private long handoffConditionTimeout = 0;
private boolean reportParseExceptions = false;
private boolean logParseExceptions = true;
@@ -278,6 +296,11 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
}
+ public KinesisIndexTaskTest(LockGranularity lockGranularity)
+ {
+ this.lockGranularity = lockGranularity;
+ }
+
@Before
public void setupTest() throws IOException, InterruptedException
{
@@ -428,9 +451,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
@@ -439,8 +462,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 120_000L)
@@ -497,9 +521,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2012/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2011/P1D", 0);
+ SegmentDescriptor desc2 = sd("2012/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "1"))
@@ -508,8 +532,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 120_000L)
@@ -601,14 +626,14 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = sd(task, "2011/P1D", 1);
- SegmentDescriptor desc6 = sd(task, "2012/P1D", 0);
- SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2010/P1D", 0);
+ SegmentDescriptor desc4 = sd("2011/P1D", 0);
+ SegmentDescriptor desc5 = sd("2011/P1D", 1);
+ SegmentDescriptor desc6 = sd("2012/P1D", 0);
+ SegmentDescriptor desc7 = sd("2013/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(
@@ -620,15 +645,16 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
- Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", desc4))
- && ImmutableList.of("h").equals(readSegmentColumn("dim1", desc5))) ||
- (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", desc4))
- && ImmutableList.of("e").equals(readSegmentColumn("dim1", desc5))));
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc6));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
+ Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", publishedDescriptors.get(3)))
+ && ImmutableList.of("h").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))) ||
+ (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", publishedDescriptors.get(3)))
+ && ImmutableList.of("e").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))));
+ Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(5)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(6)));
}
@Test(timeout = 120_000L)
@@ -751,25 +777,26 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc5 = sd(task, "2049/P1D", 0);
- SegmentDescriptor desc7 = sd(task, "2013/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2010/P1D", 0);
+ SegmentDescriptor desc4 = sd("2011/P1D", 0);
+ SegmentDescriptor desc5 = sd("2049/P1D", 0);
+ SegmentDescriptor desc7 = sd("2013/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "10"))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc4));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc5));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc7));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(3)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(4)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(5)));
}
@@ -828,17 +855,18 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@@ -897,10 +925,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2010/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2010/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
@@ -908,9 +936,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
}
@@ -977,8 +1006,8 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2009/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2009/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
@@ -986,8 +1015,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", publishedDescriptors.get(0)));
}
@@ -1043,7 +1073,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- Assert.assertEquals(ImmutableSet.of(sd(task, "2010/P1D", 0)), publishedDescriptors());
+ assertEqualsExceptVersion(ImmutableList.of(sd("2010/P1D", 0)), publishedDescriptors());
}
@@ -1099,9 +1129,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
@@ -1110,8 +1140,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@@ -1168,9 +1199,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
@@ -1179,8 +1210,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@@ -1240,7 +1272,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+ Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
}
@@ -1303,11 +1335,11 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc3 = sd(task, "2013/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2049/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ SegmentDescriptor desc3 = sd("2013/P1D", 0);
+ SegmentDescriptor desc4 = sd("2049/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "12"))
@@ -1402,7 +1434,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- Assert.assertEquals(ImmutableSet.of(), publishedDescriptors());
+ Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
@@ -1504,9 +1536,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
@@ -1515,8 +1547,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@@ -1596,17 +1629,18 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata, should all be from the first task
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@@ -1673,9 +1707,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
// Run second task
@@ -1693,16 +1727,17 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
- SegmentDescriptor desc4 = sd(task2, "2013/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+ SegmentDescriptor desc3 = sd("2011/P1D", 1);
+ SegmentDescriptor desc4 = sd("2013/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()));
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3));
- Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
+ Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(3)));
}
@@ -1765,10 +1800,10 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- SegmentDescriptor desc4 = sd(task, "2012/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ SegmentDescriptor desc4 = sd("2012/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc4), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1"))
@@ -1777,13 +1812,14 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(2)));
// Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically
Assert.assertEquals(
ImmutableSet.of(ImmutableList.of("d", "e", "h")),
- ImmutableSet.of(readSegmentColumn("dim1", desc2))
+ ImmutableSet.of(readSegmentColumn("dim1", publishedDescriptors.get(1)))
);
}
@@ -1863,12 +1899,12 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- SegmentDescriptor desc3 = sd(task2, "2011/P1D", 1);
- SegmentDescriptor desc4 = sd(task2, "2012/P1D", 0);
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ SegmentDescriptor desc3 = sd("2011/P1D", 1);
+ SegmentDescriptor desc4 = sd("2012/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors());
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1"))
@@ -1877,13 +1913,14 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
// Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically
Assert.assertEquals(
ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")),
- ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3))
+ ImmutableSet.of(readSegmentColumn("dim1", publishedDescriptors.get(1)), readSegmentColumn("dim1", publishedDescriptors.get(2)))
);
- Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4));
+ Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(3)));
}
@@ -1995,9 +2032,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "5"))),
@@ -2005,8 +2042,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 120_000L)
@@ -2137,13 +2175,13 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway());
// Check published segments & metadata
- SegmentDescriptor desc1 = sd(task1, "2008/P1D", 0);
- SegmentDescriptor desc2 = sd(task1, "2009/P1D", 0);
- SegmentDescriptor desc3 = sd(task1, "2010/P1D", 0);
- SegmentDescriptor desc4 = sd(task1, "2011/P1D", 0);
- SegmentDescriptor desc5 = sd(task1, "2012/P1D", 0);
- SegmentDescriptor desc6 = sd(task1, "2013/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc3, desc4, desc5, desc6), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2008/P1D", 0);
+ SegmentDescriptor desc2 = sd("2009/P1D", 0);
+ SegmentDescriptor desc3 = sd("2010/P1D", 0);
+ SegmentDescriptor desc4 = sd("2011/P1D", 0);
+ SegmentDescriptor desc5 = sd("2012/P1D", 0);
+ SegmentDescriptor desc6 = sd("2013/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "6"))
@@ -2239,9 +2277,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(
STREAM,
@@ -2251,8 +2289,9 @@ public class KinesisIndexTaskTest extends EasyMockSupport
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 60_000L)
@@ -2320,17 +2359,18 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- SegmentDescriptor desc1 = sd(task, "2010/P1D", 0);
- SegmentDescriptor desc2 = sd(task, "2011/P1D", 0);
- Assert.assertEquals(ImmutableSet.of(desc1, desc2), publishedDescriptors());
+ SegmentDescriptor desc1 = sd("2010/P1D", 0);
+ SegmentDescriptor desc2 = sd("2011/P1D", 0);
+ assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())
);
// Check segments in deep storage
- Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1));
- Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2));
+ final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
+ Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0)));
+ Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1)));
}
@Test(timeout = 5000L)
@@ -2468,16 +2508,16 @@ public class KinesisIndexTaskTest extends EasyMockSupport
Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getThrownAway());
// Check published metadata
- final Set<SegmentDescriptor> descriptors = new HashSet<>();
- descriptors.add(sd(normalReplica, "2008/P1D", 0));
- descriptors.add(sd(normalReplica, "2009/P1D", 0));
- descriptors.add(sd(normalReplica, "2010/P1D", 0));
- descriptors.add(sd(normalReplica, "2010/P1D", 1));
- descriptors.add(sd(normalReplica, "2011/P1D", 0));
- descriptors.add(sd(normalReplica, "2011/P1D", 1));
- descriptors.add(sd(normalReplica, "2012/P1D", 0));
- descriptors.add(sd(normalReplica, "2013/P1D", 0));
- Assert.assertEquals(descriptors, publishedDescriptors());
+ final List<SegmentDescriptor> descriptors = new ArrayList<>();
+ descriptors.add(sd("2008/P1D", 0));
+ descriptors.add(sd("2009/P1D", 0));
+ descriptors.add(sd("2010/P1D", 0));
+ descriptors.add(sd("2010/P1D", 1));
+ descriptors.add(sd("2011/P1D", 0));
+ descriptors.add(sd("2011/P1D", 1));
+ descriptors.add(sd("2012/P1D", 0));
+ descriptors.add(sd("2013/P1D", 0));
+ assertEqualsExceptVersion(descriptors, publishedDescriptors());
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "9"))
@@ -2571,6 +2611,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
return taskExec.submit(
() -> {
try {
+ task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK);
if (task.isReady(toolbox.getTaskActionClient())) {
return task.run(toolbox);
} else {
@@ -2772,7 +2813,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport
derby.metadataTablesConfigSupplier().get(),
derbyConnector
);
- taskLockbox = new TaskLockbox(taskStorage);
+ taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator);
final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
taskLockbox,
taskStorage,
@@ -2877,14 +2918,12 @@ public class KinesisIndexTaskTest extends EasyMockSupport
}
- private Set<SegmentDescriptor> publishedDescriptors()
+ private List<SegmentDescriptor> publishedDescriptors()
{
- return FluentIterable.from(
- metadataStorageCoordinator.getUsedSegmentsForInterval(
- DATA_SCHEMA.getDataSource(),
- Intervals.of("0000/3000")
- )
- ).transform(DataSegment::toDescriptor).toSet();
+ return metadataStorageCoordinator.getUsedSegmentsForInterval(
+ DATA_SCHEMA.getDataSource(),
+ Intervals.of("0000/3000")
+ ).stream().map(DataSegment::toDescriptor).collect(Collectors.toList());
}
private void unlockAppenderatorBasePersistDirForTask(KinesisIndexTask task)
@@ -2991,10 +3030,33 @@ public class KinesisIndexTaskTest extends EasyMockSupport
}
}
- private SegmentDescriptor sd(final Task task, final String intervalString, final int partitionNum)
+ private SegmentDescriptor sd(final String intervalString, final int partitionNum)
{
final Interval interval = Intervals.of(intervalString);
- return new SegmentDescriptor(interval, getLock(task, interval).getVersion(), partitionNum);
+ return new SegmentDescriptor(interval, "fakeVersion", partitionNum);
+ }
+
+ private void assertEqualsExceptVersion(List<SegmentDescriptor> descriptors1, List<SegmentDescriptor> descriptors2)
+ {
+ Assert.assertEquals(descriptors1.size(), descriptors2.size());
+ final Comparator<SegmentDescriptor> comparator = (s1, s2) -> {
+ final int intervalCompare = Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval());
+ if (intervalCompare == 0) {
+ return Integer.compare(s1.getPartitionNumber(), s2.getPartitionNumber());
+ } else {
+ return intervalCompare;
+ }
+ };
+
+ final List<SegmentDescriptor> copy1 = new ArrayList<>(descriptors1);
+ final List<SegmentDescriptor> copy2 = new ArrayList<>(descriptors2);
+ copy1.sort(comparator);
+ copy2.sort(comparator);
+
+ for (int i = 0; i < copy1.size(); i++) {
+ Assert.assertEquals(copy1.get(i).getInterval(), copy2.get(i).getInterval());
+ Assert.assertEquals(copy1.get(i).getPartitionNumber(), copy2.get(i).getPartitionNumber());
+ }
}
private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java
index f4cad31..79e6d83 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java
@@ -823,6 +823,7 @@ public class IndexGeneratorJob implements Jobby
-1,
-1
);
+
final DataSegment segment = JobHelper.serializeOutIndex(
segmentTemplate,
context.getConfiguration(),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLockType.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java
similarity index 89%
copy from indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLockType.java
copy to indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java
index e1f3f4e..37cdecc 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLockType.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java
@@ -19,8 +19,11 @@
package org.apache.druid.indexing.common;
-public enum TaskLockType
+/**
+ * Granularity for {@link TaskLock}.
+ */
+public enum LockGranularity
{
- SHARED,
- EXCLUSIVE
+ TIME_CHUNK,
+ SEGMENT
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentLock.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentLock.java
new file mode 100644
index 0000000..ec8b24e
--- /dev/null
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentLock.java
@@ -0,0 +1,233 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.indexing.overlord.LockRequest;
+import org.apache.druid.indexing.overlord.LockRequestForNewSegment;
+import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest;
+import org.apache.druid.indexing.overlord.TimeChunkLockRequest;
+import org.apache.druid.java.util.common.ISE;
+import org.joda.time.Interval;
+
+import java.util.Objects;
+
+/**
+ * Lock for set of segments. Should be unique for (dataSource, interval, version, partitionId).
+ */
+public class SegmentLock implements TaskLock
+{
+ static final String TYPE = "segment";
+
+ private final TaskLockType lockType;
+ private final String groupId;
+ private final String dataSource;
+ private final Interval interval;
+ private final String version;
+ private final int partitionId;
+ private final int priority;
+ private final boolean revoked;
+
+ @JsonCreator
+ public SegmentLock(
+ @JsonProperty("type") TaskLockType lockType,
+ @JsonProperty("groupId") String groupId,
+ @JsonProperty("dataSource") String dataSource,
+ @JsonProperty("interval") Interval interval,
+ @JsonProperty("version") String version,
+ @JsonProperty("partitionId") int partitionId,
+ @JsonProperty("priority") int priority,
+ @JsonProperty("revoked") boolean revoked
+ )
+ {
+ this.lockType = Preconditions.checkNotNull(lockType, "lockType");
+ this.groupId = Preconditions.checkNotNull(groupId, "groupId");
+ this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
+ this.interval = Preconditions.checkNotNull(interval, "interval");
+ this.version = Preconditions.checkNotNull(version, "version");
+ this.partitionId = partitionId;
+ this.priority = priority;
+ this.revoked = revoked;
+ }
+
+ public SegmentLock(
+ TaskLockType lockType,
+ String groupId,
+ String dataSource,
+ Interval interval,
+ String version,
+ int partitionId,
+ int priority
+ )
+ {
+ this(lockType, groupId, dataSource, interval, version, partitionId, priority, false);
+ }
+
+ @Override
+ public TaskLock revokedCopy()
+ {
+ return new SegmentLock(lockType, groupId, dataSource, interval, version, partitionId, priority, true);
+ }
+
+ @Override
+ public TaskLock withPriority(int newPriority)
+ {
+ return new SegmentLock(lockType, groupId, dataSource, interval, version, partitionId, newPriority, revoked);
+ }
+
+ @Override
+ public LockGranularity getGranularity()
+ {
+ return LockGranularity.SEGMENT;
+ }
+
+ @JsonProperty
+ @Override
+ public TaskLockType getType()
+ {
+ return lockType;
+ }
+
+ @JsonProperty
+ @Override
+ public String getGroupId()
+ {
+ return groupId;
+ }
+
+ @JsonProperty
+ @Override
+ public String getDataSource()
+ {
+ return dataSource;
+ }
+
+ @JsonProperty
+ @Override
+ public Interval getInterval()
+ {
+ return interval;
+ }
+
+ @JsonProperty
+ public int getPartitionId()
+ {
+ return partitionId;
+ }
+
+ @JsonProperty
+ @Override
+ public String getVersion()
+ {
+ return version;
+ }
+
+ @JsonProperty
+ @Override
+ public Integer getPriority()
+ {
+ return priority;
+ }
+
+ @Override
+ public int getNonNullPriority()
+ {
+ return priority;
+ }
+
+ @JsonProperty
+ @Override
+ public boolean isRevoked()
+ {
+ return revoked;
+ }
+
+ @Override
+ public boolean conflict(LockRequest request)
+ {
+ if (request instanceof LockRequestForNewSegment) {
+ // request for new segments doens't conflict with any locks because it allocates a new partitionId
+ return false;
+ }
+
+ if (!dataSource.equals(request.getDataSource())) {
+ return false;
+ }
+
+ if (request instanceof TimeChunkLockRequest) {
+ // For different interval, all overlapping intervals cause conflict.
+ return interval.overlaps(request.getInterval());
+ } else if (request instanceof SpecificSegmentLockRequest) {
+ if (interval.equals(request.getInterval())) {
+ final SpecificSegmentLockRequest specificSegmentLockRequest = (SpecificSegmentLockRequest) request;
+ // Lock conflicts only if the interval is same and the partitionIds intersect.
+ return specificSegmentLockRequest.getPartitionId() == partitionId;
+ } else {
+ // For different interval, all overlapping intervals cause conflict.
+ return interval.overlaps(request.getInterval());
+ }
+ } else {
+ throw new ISE("Unknown request type[%s]", request.getClass().getName());
+ }
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ SegmentLock that = (SegmentLock) o;
+ return partitionId == that.partitionId &&
+ priority == that.priority &&
+ revoked == that.revoked &&
+ lockType == that.lockType &&
+ Objects.equals(groupId, that.groupId) &&
+ Objects.equals(dataSource, that.dataSource) &&
+ Objects.equals(interval, that.interval) &&
+ Objects.equals(version, that.version);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(lockType, groupId, dataSource, interval, partitionId, version, priority, revoked);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "SegmentLock{" +
+ "lockType=" + lockType +
+ ", groupId='" + groupId + '\'' +
+ ", dataSource='" + dataSource + '\'' +
+ ", interval=" + interval +
+ ", version='" + version + '\'' +
+ ", partitionId=" + partitionId +
+ ", priority=" + priority +
+ ", revoked=" + revoked +
+ '}';
+ }
+}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLock.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLock.java
index 865b053..8c1e5d2 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLock.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLock.java
@@ -19,10 +19,11 @@
package org.apache.druid.indexing.common;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
+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.Id;
+import org.apache.druid.indexing.overlord.LockRequest;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@@ -30,156 +31,35 @@ import javax.annotation.Nullable;
/**
* Represents a lock held by some task. Immutable.
*/
-public class TaskLock
+@JsonTypeInfo(use = Id.NAME, property = "granularity", defaultImpl = TimeChunkLock.class)
+@JsonSubTypes(value = {
+ @Type(name = TimeChunkLock.TYPE, value = TimeChunkLock.class),
+ @Type(name = SegmentLock.TYPE, value = SegmentLock.class)
+})
+public interface TaskLock
{
- private final TaskLockType type;
- private final String groupId;
- private final String dataSource;
- private final Interval interval;
- private final String version;
- private final Integer priority;
- private final boolean revoked;
+ TaskLock revokedCopy();
- public static TaskLock withPriority(TaskLock lock, int priority)
- {
- return new TaskLock(
- lock.type,
- lock.getGroupId(),
- lock.getDataSource(),
- lock.getInterval(),
- lock.getVersion(),
- priority,
- lock.isRevoked()
- );
- }
+ TaskLock withPriority(int priority);
- @JsonCreator
- public TaskLock(
- @JsonProperty("type") @Nullable TaskLockType type, // nullable for backward compatibility
- @JsonProperty("groupId") String groupId,
- @JsonProperty("dataSource") String dataSource,
- @JsonProperty("interval") Interval interval,
- @JsonProperty("version") String version,
- @JsonProperty("priority") @Nullable Integer priority,
- @JsonProperty("revoked") boolean revoked
- )
- {
- this.type = type == null ? TaskLockType.EXCLUSIVE : type;
- this.groupId = Preconditions.checkNotNull(groupId, "groupId");
- this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
- this.interval = Preconditions.checkNotNull(interval, "interval");
- this.version = Preconditions.checkNotNull(version, "version");
- this.priority = priority;
- this.revoked = revoked;
- }
+ LockGranularity getGranularity();
- public TaskLock(
- TaskLockType type,
- String groupId,
- String dataSource,
- Interval interval,
- String version,
- int priority
- )
- {
- this(type, groupId, dataSource, interval, version, priority, false);
- }
+ TaskLockType getType();
- public TaskLock revokedCopy()
- {
- return new TaskLock(
- type,
- groupId,
- dataSource,
- interval,
- version,
- priority,
- true
- );
- }
+ String getGroupId();
- @JsonProperty
- public TaskLockType getType()
- {
- return type;
- }
+ String getDataSource();
- @JsonProperty
- public String getGroupId()
- {
- return groupId;
- }
+ Interval getInterval();
- @JsonProperty
- public String getDataSource()
- {
- return dataSource;
- }
+ String getVersion();
- @JsonProperty
- public Interval getInterval()
- {
- return interval;
- }
-
- @JsonProperty
- public String getVersion()
- {
- return version;
- }
-
- @JsonProperty
@Nullable
- public Integer getPriority()
- {
- return priority;
- }
-
- public int getNonNullPriority()
- {
- return Preconditions.checkNotNull(priority, "priority");
- }
-
- @JsonProperty
- public boolean isRevoked()
- {
- return revoked;
- }
+ Integer getPriority();
- @Override
- public boolean equals(Object o)
- {
- if (!(o instanceof TaskLock)) {
- return false;
- } else {
- final TaskLock that = (TaskLock) o;
- return this.type.equals(that.type) &&
- this.groupId.equals(that.groupId) &&
- this.dataSource.equals(that.dataSource) &&
- this.interval.equals(that.interval) &&
- this.version.equals(that.version) &&
- Objects.equal(this.priority, that.priority) &&
- this.revoked == that.revoked;
- }
- }
+ int getNonNullPriority();
- @Override
- public int hashCode()
- {
- return Objects.hashCode(type, groupId, dataSource, interval, version, priority, revoked);
- }
+ boolean isRevoked();
- @Override
- public String toString()
- {
- return Objects.toStringHelper(this)
- .add("type", type)
- .add("groupId", groupId)
- .add("dataSource", dataSource)
- .add("interval", interval)
- .add("version", version)
- .add("priority", priority)
- .add("revoked", revoked)
- .toString();
- }
+ boolean conflict(LockRequest request);
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLockType.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLockType.java
index e1f3f4e..b51990c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLockType.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLockType.java
@@ -22,5 +22,5 @@ package org.apache.druid.indexing.common;
public enum TaskLockType
{
SHARED,
- EXCLUSIVE
+ EXCLUSIVE // taskLocks of this type can be shared by tasks of the same groupId.
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLock.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java
similarity index 60%
copy from indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLock.java
copy to indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java
index 865b053..15bb3ff 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskLock.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java
@@ -21,41 +21,30 @@ package org.apache.druid.indexing.common;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Objects;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import org.apache.druid.indexing.overlord.LockRequest;
import org.joda.time.Interval;
import javax.annotation.Nullable;
+import java.util.Objects;
-/**
- * Represents a lock held by some task. Immutable.
- */
-public class TaskLock
+public class TimeChunkLock implements TaskLock
{
- private final TaskLockType type;
+ static final String TYPE = "timeChunk";
+
+ private final TaskLockType lockType;
private final String groupId;
private final String dataSource;
private final Interval interval;
private final String version;
+ @Nullable
private final Integer priority;
private final boolean revoked;
- public static TaskLock withPriority(TaskLock lock, int priority)
- {
- return new TaskLock(
- lock.type,
- lock.getGroupId(),
- lock.getDataSource(),
- lock.getInterval(),
- lock.getVersion(),
- priority,
- lock.isRevoked()
- );
- }
-
@JsonCreator
- public TaskLock(
- @JsonProperty("type") @Nullable TaskLockType type, // nullable for backward compatibility
+ public TimeChunkLock(
+ @JsonProperty("type") @Nullable TaskLockType lockType, // nullable for backward compatibility
@JsonProperty("groupId") String groupId,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval,
@@ -64,7 +53,7 @@ public class TaskLock
@JsonProperty("revoked") boolean revoked
)
{
- this.type = type == null ? TaskLockType.EXCLUSIVE : type;
+ this.lockType = lockType == null ? TaskLockType.EXCLUSIVE : lockType;
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
this.interval = Preconditions.checkNotNull(interval, "interval");
@@ -73,7 +62,8 @@ public class TaskLock
this.revoked = revoked;
}
- public TaskLock(
+ @VisibleForTesting
+ public TimeChunkLock(
TaskLockType type,
String groupId,
String dataSource,
@@ -85,10 +75,11 @@ public class TaskLock
this(type, groupId, dataSource, interval, version, priority, false);
}
+ @Override
public TaskLock revokedCopy()
{
- return new TaskLock(
- type,
+ return new TimeChunkLock(
+ lockType,
groupId,
dataSource,
interval,
@@ -98,36 +89,62 @@ public class TaskLock
);
}
+ @Override
+ public TaskLock withPriority(int priority)
+ {
+ return new TimeChunkLock(
+ this.lockType,
+ this.groupId,
+ this.dataSource,
+ this.interval,
+ this.version,
+ priority,
+ this.revoked
+ );
+ }
+
+ @Override
+ public LockGranularity getGranularity()
+ {
+ return LockGranularity.TIME_CHUNK;
+ }
+
+ @Override
@JsonProperty
public TaskLockType getType()
{
- return type;
+ return lockType;
}
+ @Override
@JsonProperty
public String getGroupId()
{
return groupId;
}
+ @Override
@JsonProperty
public String getDataSource()
{
return dataSource;
}
+ @Override
@JsonProperty
public Interval getInterval()
{
return interval;
}
+ @Override
@JsonProperty
public String getVersion()
{
return version;
}
+ @Override
@JsonProperty
@Nullable
public Integer getPriority()
@@ -135,11 +152,13 @@ public class TaskLock
return priority;
}
+ @Override
public int getNonNullPriority()
{
return Preconditions.checkNotNull(priority, "priority");
}
+ @Override
@JsonProperty
public boolean isRevoked()
{
@@ -147,39 +166,48 @@ public class TaskLock
}
@Override
+ public boolean conflict(LockRequest request)
+ {
+ return dataSource.equals(request.getDataSource())
+ && interval.overlaps(request.getInterval());
+ }
+
+ @Override
public boolean equals(Object o)
{
- if (!(o instanceof TaskLock)) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
return false;
- } else {
- final TaskLock that = (TaskLock) o;
- return this.type.equals(that.type) &&
- this.groupId.equals(that.groupId) &&
- this.dataSource.equals(that.dataSource) &&
- this.interval.equals(that.interval) &&
- this.version.equals(that.version) &&
- Objects.equal(this.priority, that.priority) &&
- this.revoked == that.revoked;
}
+ TimeChunkLock that = (TimeChunkLock) o;
+ return revoked == that.revoked &&
+ lockType == that.lockType &&
+ Objects.equals(groupId, that.groupId) &&
+ Objects.equals(dataSource, that.dataSource) &&
+ Objects.equals(interval, that.interval) &&
+ Objects.equals(version, that.version) &&
+ Objects.equals(priority, that.priority);
}
@Override
public int hashCode()
{
- return Objects.hashCode(type, groupId, dataSource, interval, version, priority, revoked);
+ return Objects.hash(lockType, groupId, dataSource, interval, version, priority, revoked);
}
@Override
public String toString()
{
- return Objects.toStringHelper(this)
- .add("type", type)
- .add("groupId", groupId)
- .add("dataSource", dataSource)
- .add("interval", interval)
- .add("version", version)
- .add("priority", priority)
- .add("revoked", revoked)
- .toString();
+ return "TimeChunkLock{" +
+ "type=" + lockType +
+ ", groupId='" + groupId + '\'' +
+ ", dataSource='" + dataSource + '\'' +
+ ", interval=" + interval +
+ ", version='" + version + '\'' +
+ ", priority=" + priority +
+ ", revoked=" + revoked +
+ '}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java
index 97959d7..9b2a765 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LocalTaskActionClient.java
@@ -19,6 +19,7 @@
package org.apache.druid.indexing.common.actions;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.TaskStorage;
@@ -26,15 +27,20 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
public class LocalTaskActionClient implements TaskActionClient
{
+ private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
+
+ private final ConcurrentHashMap<Class<? extends TaskAction>, AtomicInteger> actionCountMap = new ConcurrentHashMap<>();
+
private final Task task;
private final TaskStorage storage;
private final TaskActionToolbox toolbox;
private final TaskAuditLogConfig auditLogConfig;
- private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
-
public LocalTaskActionClient(
Task task,
TaskStorage storage,
@@ -73,9 +79,17 @@ public class LocalTaskActionClient implements TaskActionClient
final long performStartTime = System.currentTimeMillis();
final RetType result = taskAction.perform(task, toolbox);
emitTimerMetric("task/action/run/time", System.currentTimeMillis() - performStartTime);
+ actionCountMap.computeIfAbsent(taskAction.getClass(), k -> new AtomicInteger()).incrementAndGet();
return result;
}
+ @VisibleForTesting
+ public int getActionCount(Class<? extends TaskAction> actionClass)
+ {
+ final AtomicInteger count = actionCountMap.get(actionClass);
+ return count == null ? 0 : count.get();
+ }
+
private void emitTimerMetric(final String metric, final long time)
{
final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent.builder();
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
index 6623483..aecfcec 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
@@ -19,15 +19,16 @@
package org.apache.druid.indexing.common.actions;
+import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
+import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.CriticalAction;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.LockRequestForNewSegment;
import org.apache.druid.indexing.overlord.LockResult;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
@@ -36,9 +37,13 @@ import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
+import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.joda.time.DateTime;
import org.joda.time.Interval;
+import javax.annotation.Nullable;
+import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
@@ -49,10 +54,12 @@ import java.util.stream.Collectors;
* segments for the given timestamp, or if the prior segments for the given timestamp are already at the
* preferredSegmentGranularity. Otherwise, the prior segments will take precedence.
* <p/>
- * This action implicitly acquires locks when it allocates segments. You do not have to acquire them beforehand,
- * although you *do* have to release them yourself.
+ * This action implicitly acquires some task locks when it allocates segments. You do not have to acquire them
+ * beforehand, although you *do* have to release them yourself. (Note that task locks are automatically released when
+ * the task is finished.)
* <p/>
- * If this action cannot acquire an appropriate lock, or if it cannot expand an existing segment set, it returns null.
+ * If this action cannot acquire an appropriate task lock, or if it cannot expand an existing segment set, it returns
+ * null.
*/
public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
{
@@ -68,7 +75,10 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
private final String sequenceName;
private final String previousSegmentId;
private final boolean skipSegmentLineageCheck;
+ private final ShardSpecFactory shardSpecFactory;
+ private final LockGranularity lockGranularity;
+ @JsonCreator
public SegmentAllocateAction(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("timestamp") DateTime timestamp,
@@ -76,7 +86,9 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
@JsonProperty("preferredSegmentGranularity") Granularity preferredSegmentGranularity,
@JsonProperty("sequenceName") String sequenceName,
@JsonProperty("previousSegmentId") String previousSegmentId,
- @JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck
+ @JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck,
+ @JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory, // nullable for backward compatibility
+ @JsonProperty("lockGranularity") @Nullable LockGranularity lockGranularity // nullable for backward compatibility
)
{
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
@@ -89,6 +101,8 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName");
this.previousSegmentId = previousSegmentId;
this.skipSegmentLineageCheck = skipSegmentLineageCheck;
+ this.shardSpecFactory = shardSpecFactory == null ? NumberedShardSpecFactory.instance() : shardSpecFactory;
+ this.lockGranularity = lockGranularity == null ? LockGranularity.TIME_CHUNK : lockGranularity;
}
@JsonProperty
@@ -133,6 +147,18 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
return skipSegmentLineageCheck;
}
+ @JsonProperty
+ public ShardSpecFactory getShardSpecFactory()
+ {
+ return shardSpecFactory;
+ }
+
+ @JsonProperty
+ public LockGranularity getLockGranularity()
+ {
+ return lockGranularity;
+ }
+
@Override
public TypeReference<SegmentIdWithShardSpec> getReturnTypeReference()
{
@@ -162,18 +188,18 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
final Interval rowInterval = queryGranularity.bucket(timestamp);
- final Set<DataSegment> usedSegmentsForRow = ImmutableSet.copyOf(
+ final Set<DataSegment> usedSegmentsForRow = new HashSet<>(
msc.getUsedSegmentsForInterval(dataSource, rowInterval)
);
final SegmentIdWithShardSpec identifier = usedSegmentsForRow.isEmpty() ?
tryAllocateFirstSegment(toolbox, task, rowInterval) :
tryAllocateSubsequentSegment(
- toolbox,
- task,
- rowInterval,
- usedSegmentsForRow.iterator().next()
- );
+ toolbox,
+ task,
+ rowInterval,
+ usedSegmentsForRow.iterator().next()
+ );
if (identifier != null) {
return identifier;
}
@@ -257,43 +283,31 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
boolean logOnFail
)
{
- log.debug(
- "Trying to allocate pending segment for rowInterval[%s], segmentInterval[%s].",
- rowInterval,
- tryInterval
+ // This action is always used by appending tasks, which cannot change the segment granularity of existing
+ // dataSources. So, all lock requests should be segmentLock.
+ final LockResult lockResult = toolbox.getTaskLockbox().tryLock(
+ task,
+ new LockRequestForNewSegment(
+ lockGranularity,
+ TaskLockType.EXCLUSIVE,
+ task.getGroupId(),
+ dataSource,
+ tryInterval,
+ shardSpecFactory,
+ task.getPriority(),
+ sequenceName,
+ previousSegmentId,
+ skipSegmentLineageCheck
+ )
);
- final LockResult lockResult = toolbox.getTaskLockbox().tryLock(TaskLockType.EXCLUSIVE, task, tryInterval);
+
if (lockResult.isRevoked()) {
// We had acquired a lock but it was preempted by other locks
throw new ISE("The lock for interval[%s] is preempted and no longer valid", tryInterval);
}
if (lockResult.isOk()) {
- final SegmentIdWithShardSpec identifier;
- try {
- identifier = toolbox.getTaskLockbox().doInCriticalSection(
- task,
- ImmutableList.of(tryInterval),
- CriticalAction
- .<SegmentIdWithShardSpec>builder()
- .onValidLocks(
- () -> toolbox.getIndexerMetadataStorageCoordinator().allocatePendingSegment(
- dataSource,
- sequenceName,
- previousSegmentId,
- tryInterval,
- lockResult.getTaskLock().getVersion(),
- skipSegmentLineageCheck
- )
- )
- .onInvalidLocks(() -> null)
- .build()
- );
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
-
+ final SegmentIdWithShardSpec identifier = lockResult.getNewSegmentId();
if (identifier != null) {
return identifier;
} else {
@@ -340,7 +354,9 @@ public class SegmentAllocateAction implements TaskAction<SegmentIdWithShardSpec>
", preferredSegmentGranularity=" + preferredSegmentGranularity +
", sequenceName='" + sequenceName + '\'' +
", previousSegmentId='" + previousSegmentId + '\'' +
- ", skipSegmentLineageCheck='" + skipSegmentLineageCheck + '\'' +
+ ", skipSegmentLineageCheck=" + skipSegmentLineageCheck +
+ ", shardSpecFactory=" + shardSpecFactory +
+ ", lockGranularity=" + lockGranularity +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
index f424c7c..e780456 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
@@ -71,7 +71,7 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
@Override
public Set<DataSegment> perform(Task task, TaskActionToolbox toolbox)
{
- return new SegmentTransactionalInsertAction(segments, null, null).perform(task, toolbox).getSegments();
+ return SegmentTransactionalInsertAction.appendAction(segments, null, null).perform(task, toolbox).getSegments();
}
@Override
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUsedAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUsedAction.java
index 50c22d3..461db0a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUsedAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUsedAction.java
@@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
+import java.util.Collection;
import java.util.List;
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
@@ -44,7 +45,7 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
public SegmentListUsedAction(
@JsonProperty("dataSource") String dataSource,
@Deprecated @JsonProperty("interval") Interval interval,
- @JsonProperty("intervals") List<Interval> intervals
+ @JsonProperty("intervals") Collection<Interval> intervals
)
{
this.dataSource = dataSource;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java
similarity index 53%
copy from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java
copy to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java
index 0b2e49d..70c8122 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java
@@ -20,44 +20,50 @@
package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Preconditions;
-import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.LockResult;
+import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest;
import org.joda.time.Interval;
-import javax.annotation.Nullable;
-
-public class LockAcquireAction implements TaskAction<TaskLock>
+/**
+ * TaskAction to acquire a {@link org.apache.druid.indexing.common.SegmentLock}.
+ * This action is a blocking operation and the caller could wait until it gets {@link LockResult}
+ * (up to timeoutMs if it's > 0).
+ *
+ * This action is currently used by only stream ingestion tasks.
+ */
+public class SegmentLockAcquireAction implements TaskAction<LockResult>
{
- private final TaskLockType type;
-
- @JsonIgnore
+ private final TaskLockType lockType;
private final Interval interval;
-
- @JsonIgnore
+ private final String version;
+ private final int partitionId;
private final long timeoutMs;
@JsonCreator
- public LockAcquireAction(
- @JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility
+ public SegmentLockAcquireAction(
+ @JsonProperty("lockType") TaskLockType lockType,
@JsonProperty("interval") Interval interval,
+ @JsonProperty("version") String version,
+ @JsonProperty("partitionId") int partitionId,
@JsonProperty("timeoutMs") long timeoutMs
)
{
- this.type = type == null ? TaskLockType.EXCLUSIVE : type;
+ this.lockType = Preconditions.checkNotNull(lockType, "lockType");
this.interval = Preconditions.checkNotNull(interval, "interval");
+ this.version = Preconditions.checkNotNull(version, "version");
+ this.partitionId = partitionId;
this.timeoutMs = timeoutMs;
}
- @JsonProperty("lockType")
- public TaskLockType getType()
+ @JsonProperty
+ public TaskLockType getLockType()
{
- return type;
+ return lockType;
}
@JsonProperty
@@ -67,27 +73,46 @@ public class LockAcquireAction implements TaskAction<TaskLock>
}
@JsonProperty
+ public String getVersion()
+ {
+ return version;
+ }
+
+ @JsonProperty
+ public int getPartitionId()
+ {
+ return partitionId;
+ }
+
+ @JsonProperty
public long getTimeoutMs()
{
return timeoutMs;
}
@Override
- public TypeReference<TaskLock> getReturnTypeReference()
+ public TypeReference<LockResult> getReturnTypeReference()
{
- return new TypeReference<TaskLock>()
+ return new TypeReference<LockResult>()
{
};
}
@Override
- public TaskLock perform(Task task, TaskActionToolbox toolbox)
+ public LockResult perform(Task task, TaskActionToolbox toolbox)
{
try {
- final LockResult result = timeoutMs == 0 ?
- toolbox.getTaskLockbox().lock(type, task, interval) :
- toolbox.getTaskLockbox().lock(type, task, interval, timeoutMs);
- return result.isOk() ? result.getTaskLock() : null;
+ if (timeoutMs == 0) {
+ return toolbox.getTaskLockbox().lock(
+ task,
+ new SpecificSegmentLockRequest(lockType, task, interval, version, partitionId)
+ );
+ } else {
+ return toolbox.getTaskLockbox().lock(
+ task,
+ new SpecificSegmentLockRequest(lockType, task, interval, version, partitionId), timeoutMs
+ );
+ }
}
catch (InterruptedException e) {
throw new RuntimeException(e);
@@ -103,9 +128,11 @@ public class LockAcquireAction implements TaskAction<TaskLock>
@Override
public String toString()
{
- return "LockAcquireAction{" +
- "lockType=" + type +
+ return "SegmentLockAcquireAction{" +
+ "lockType=" + lockType +
", interval=" + interval +
+ ", version='" + version + '\'' +
+ ", partitionId=" + partitionId +
", timeoutMs=" + timeoutMs +
'}';
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java
similarity index 57%
copy from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockTryAcquireAction.java
copy to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java
index f24266b..6a47091 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockTryAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java
@@ -20,60 +20,52 @@
package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
-import org.apache.druid.indexing.common.TaskLock;
-import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.LockResult;
import org.joda.time.Interval;
-import javax.annotation.Nullable;
-
-public class LockTryAcquireAction implements TaskAction<TaskLock>
+/**
+ * TaskAction to release a {@link org.apache.druid.indexing.common.SegmentLock}.
+ * Used by batch tasks when they fail to acquire all necessary locks.
+ */
+public class SegmentLockReleaseAction implements TaskAction<Void>
{
- @JsonIgnore
- private final TaskLockType type;
-
- @JsonIgnore
private final Interval interval;
+ private final int partitionId;
@JsonCreator
- public LockTryAcquireAction(
- @JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility
- @JsonProperty("interval") Interval interval
- )
+ public SegmentLockReleaseAction(@JsonProperty Interval interval, @JsonProperty int partitionId)
{
- this.type = type == null ? TaskLockType.EXCLUSIVE : type;
this.interval = interval;
+ this.partitionId = partitionId;
}
- @JsonProperty("lockType")
- public TaskLockType getType()
+ @JsonProperty
+ public Interval getInterval()
{
- return type;
+ return interval;
}
@JsonProperty
- public Interval getInterval()
+ public int getPartitionId()
{
- return interval;
+ return partitionId;
}
@Override
- public TypeReference<TaskLock> getReturnTypeReference()
+ public TypeReference<Void> getReturnTypeReference()
{
- return new TypeReference<TaskLock>()
+ return new TypeReference<Void>()
{
};
}
@Override
- public TaskLock perform(Task task, TaskActionToolbox toolbox)
+ public Void perform(Task task, TaskActionToolbox toolbox)
{
- final LockResult result = toolbox.getTaskLockbox().tryLock(type, task, interval);
- return result.isOk() ? result.getTaskLock() : null;
+ toolbox.getTaskLockbox().unlock(task, interval, partitionId);
+ return null;
}
@Override
@@ -85,9 +77,9 @@ public class LockTryAcquireAction implements TaskAction<TaskLock>
@Override
public String toString()
{
- return "LockTryAcquireAction{" +
- "lockType=" + type +
- ", interval=" + interval +
+ return "SegmentLockReleaseAction{" +
+ "interval=" + interval +
+ ", partitionId=" + partitionId +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java
similarity index 51%
copy from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java
copy to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java
index 0b2e49d..7728574 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java
@@ -20,38 +20,43 @@
package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Preconditions;
-import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.LockResult;
+import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest;
import org.joda.time.Interval;
-import javax.annotation.Nullable;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
-public class LockAcquireAction implements TaskAction<TaskLock>
+/**
+ * TaskAction to try to acquire a {@link org.apache.druid.indexing.common.SegmentLock}.
+ * This action returns immediately failed {@link LockResult} if it fails to get locks for the given partitionIds.
+ */
+public class SegmentLockTryAcquireAction implements TaskAction<List<LockResult>>
{
private final TaskLockType type;
-
- @JsonIgnore
private final Interval interval;
-
- @JsonIgnore
- private final long timeoutMs;
+ private final String version;
+ private final Set<Integer> partitionIds;
@JsonCreator
- public LockAcquireAction(
- @JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility
+ public SegmentLockTryAcquireAction(
+ @JsonProperty("lockType") TaskLockType type,
@JsonProperty("interval") Interval interval,
- @JsonProperty("timeoutMs") long timeoutMs
+ @JsonProperty("version") String version,
+ @JsonProperty("partitionIds") Set<Integer> partitionIds
)
{
- this.type = type == null ? TaskLockType.EXCLUSIVE : type;
+ Preconditions.checkState(partitionIds != null && !partitionIds.isEmpty(), "partitionIds is empty");
+ this.type = Preconditions.checkNotNull(type, "type");
this.interval = Preconditions.checkNotNull(interval, "interval");
- this.timeoutMs = timeoutMs;
+ this.version = Preconditions.checkNotNull(version, "version");
+ this.partitionIds = partitionIds;
}
@JsonProperty("lockType")
@@ -67,31 +72,34 @@ public class LockAcquireAction implements TaskAction<TaskLock>
}
@JsonProperty
- public long getTimeoutMs()
+ public String getVersion()
+ {
+ return version;
+ }
+
+ @JsonProperty
+ public Set<Integer> getPartitionIds()
{
- return timeoutMs;
+ return partitionIds;
}
@Override
- public TypeReference<TaskLock> getReturnTypeReference()
+ public TypeReference<List<LockResult>> getReturnTypeReference()
{
- return new TypeReference<TaskLock>()
+ return new TypeReference<List<LockResult>>()
{
};
}
@Override
- public TaskLock perform(Task task, TaskActionToolbox toolbox)
+ public List<LockResult> perform(Task task, TaskActionToolbox toolbox)
{
- try {
- final LockResult result = timeoutMs == 0 ?
- toolbox.getTaskLockbox().lock(type, task, interval) :
- toolbox.getTaskLockbox().lock(type, task, interval, timeoutMs);
- return result.isOk() ? result.getTaskLock() : null;
- }
- catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
+ return partitionIds.stream()
+ .map(partitionId -> toolbox.getTaskLockbox().tryLock(
+ task,
+ new SpecificSegmentLockRequest(type, task, interval, version, partitionId)
+ ))
+ .collect(Collectors.toList());
}
@Override
@@ -103,10 +111,11 @@ public class LockAcquireAction implements TaskAction<TaskLock>
@Override
public String toString()
{
- return "LockAcquireAction{" +
- "lockType=" + type +
+ return "SegmentLockTryAcquireAction{" +
+ "type=" + type +
", interval=" + interval +
- ", timeoutMs=" + timeoutMs +
+ ", version='" + version + '\'' +
+ ", partitionIds=" + partitionIds +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
index 446ee11..83e8bb9 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
@@ -32,9 +32,7 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
-import org.joda.time.Interval;
-import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
@@ -66,14 +64,12 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
@Override
public Void perform(Task task, TaskActionToolbox toolbox)
{
- TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments);
-
- final List<Interval> intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList());
+ TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments);
try {
toolbox.getTaskLockbox().doInCriticalSection(
task,
- intervals,
+ segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()),
CriticalAction.builder()
.onValidLocks(
() -> {
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
index f81028e..4197a21 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
@@ -32,9 +32,7 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
-import org.joda.time.Interval;
-import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
@@ -68,14 +66,12 @@ public class SegmentNukeAction implements TaskAction<Void>
@Override
public Void perform(Task task, TaskActionToolbox toolbox)
{
- TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments);
-
- final List<Interval> intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList());
+ TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments);
try {
toolbox.getTaskLockbox().doInCriticalSection(
task,
- intervals,
+ segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()),
CriticalAction.builder()
.onValidLocks(
() -> {
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
index ca595a0..e1428c6 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
@@ -22,16 +22,28 @@ package org.apache.druid.indexing.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
+import org.apache.druid.indexing.common.LockGranularity;
+import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
+import org.apache.druid.indexing.common.task.SegmentLockHelper;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.SegmentPublishResult;
+import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.timeline.DataSegment;
+import org.joda.time.Interval;
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
@@ -45,43 +57,67 @@ import java.util.stream.Collectors;
*/
public class SegmentTransactionalInsertAction implements TaskAction<SegmentPublishResult>
{
-
+ @Nullable
+ private final Set<DataSegment> segmentsToBeOverwritten;
private final Set<DataSegment> segments;
+ @Nullable
private final DataSourceMetadata startMetadata;
+ @Nullable
private final DataSourceMetadata endMetadata;
- public SegmentTransactionalInsertAction(
- Set<DataSegment> segments
+ public static SegmentTransactionalInsertAction overwriteAction(
+ @Nullable Set<DataSegment> segmentsToBeOverwritten,
+ Set<DataSegment> segmentsToPublish
+ )
+ {
+ return new SegmentTransactionalInsertAction(segmentsToBeOverwritten, segmentsToPublish, null, null);
+ }
+
+ public static SegmentTransactionalInsertAction appendAction(
+ Set<DataSegment> segments,
+ @Nullable DataSourceMetadata startMetadata,
+ @Nullable DataSourceMetadata endMetadata
)
{
- this(segments, null, null);
+ return new SegmentTransactionalInsertAction(null, segments, startMetadata, endMetadata);
}
@JsonCreator
- public SegmentTransactionalInsertAction(
+ private SegmentTransactionalInsertAction(
+ @JsonProperty("segmentsToBeOverwritten") @Nullable Set<DataSegment> segmentsToBeOverwritten,
@JsonProperty("segments") Set<DataSegment> segments,
- @JsonProperty("startMetadata") DataSourceMetadata startMetadata,
- @JsonProperty("endMetadata") DataSourceMetadata endMetadata
+ @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata,
+ @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata
)
{
+ this.segmentsToBeOverwritten = segmentsToBeOverwritten;
this.segments = ImmutableSet.copyOf(segments);
this.startMetadata = startMetadata;
this.endMetadata = endMetadata;
}
@JsonProperty
+ @Nullable
+ public Set<DataSegment> getSegmentsToBeOverwritten()
+ {
+ return segmentsToBeOverwritten;
+ }
+
+ @JsonProperty
public Set<DataSegment> getSegments()
{
return segments;
}
@JsonProperty
+ @Nullable
public DataSourceMetadata getStartMetadata()
{
return startMetadata;
}
@JsonProperty
+ @Nullable
public DataSourceMetadata getEndMetadata()
{
return endMetadata;
@@ -96,19 +132,30 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
}
/**
- * Behaves similarly to
- * {@link org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#announceHistoricalSegments(Set, DataSourceMetadata, DataSourceMetadata)}.
+ * Performs some sanity checks and publishes the given segments.
*/
@Override
public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox)
{
- TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments);
+ final Set<DataSegment> allSegments = new HashSet<>(segments);
+ if (segmentsToBeOverwritten != null) {
+ allSegments.addAll(segmentsToBeOverwritten);
+ }
+ TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments);
+
+ if (segmentsToBeOverwritten != null && !segmentsToBeOverwritten.isEmpty()) {
+ final List<TaskLock> locks = toolbox.getTaskLockbox().findLocksForTask(task);
+ // Let's do some sanity check that newSegments can overwrite oldSegments.
+ if (locks.get(0).getGranularity() == LockGranularity.SEGMENT) {
+ checkWithSegmentLock();
+ }
+ }
final SegmentPublishResult retVal;
try {
retVal = toolbox.getTaskLockbox().doInCriticalSection(
task,
- segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()),
+ allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()),
CriticalAction.<SegmentPublishResult>builder()
.onValidLocks(
() -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments(
@@ -149,6 +196,67 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
return retVal;
}
+ private void checkWithSegmentLock()
+ {
+ final Map<Interval, List<DataSegment>> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten);
+ final Map<Interval, List<DataSegment>> newSegmentsMap = groupSegmentsByIntervalAndSort(segments);
+
+ oldSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
+ newSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull);
+
+ oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> {
+ final List<DataSegment> newSegmentsPerInterval = Preconditions.checkNotNull(
+ newSegmentsMap.get(interval),
+ "segments of interval[%s]",
+ interval
+ );
+ // These lists are already sorted in groupSegmentsByIntervalAndSort().
+ final int oldStartRootPartitionId = oldSegmentsPerInterval.get(0).getStartRootPartitionId();
+ final int oldEndRootPartitionId = oldSegmentsPerInterval.get(oldSegmentsPerInterval.size() - 1)
+ .getEndRootPartitionId();
+ final int newStartRootPartitionId = newSegmentsPerInterval.get(0).getStartRootPartitionId();
+ final int newEndRootPartitionId = newSegmentsPerInterval.get(newSegmentsPerInterval.size() - 1)
+ .getEndRootPartitionId();
+
+ if (oldStartRootPartitionId != newStartRootPartitionId || oldEndRootPartitionId != newEndRootPartitionId) {
+ throw new ISE(
+ "Root partition range[%d, %d] of new segments doesn't match to root partition range[%d, %d] of old segments",
+ newStartRootPartitionId,
+ newEndRootPartitionId,
+ oldStartRootPartitionId,
+ oldEndRootPartitionId
+ );
+ }
+
+ newSegmentsPerInterval
+ .forEach(eachNewSegment -> oldSegmentsPerInterval
+ .forEach(eachOldSegment -> {
+ if (eachNewSegment.getMinorVersion() <= eachOldSegment.getMinorVersion()) {
+ throw new ISE(
+ "New segment[%s] have a smaller minor version than old segment[%s]",
+ eachNewSegment,
+ eachOldSegment
+ );
+ }
+ }));
+ });
+ }
+
+ private static Map<Interval, List<DataSegment>> groupSegmentsByIntervalAndSort(Set<DataSegment> segments)
+ {
+ final Map<Interval, List<DataSegment>> segmentsMap = new HashMap<>();
+ segments.forEach(segment -> segmentsMap.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>())
+ .add(segment));
+ segmentsMap.values().forEach(segmentsPerInterval -> segmentsPerInterval.sort((s1, s2) -> {
+ if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) {
+ return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId());
+ } else {
+ return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId());
+ }
+ }));
+ return segmentsMap;
+ }
+
@Override
public boolean isAudited()
{
@@ -159,7 +267,8 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
public String toString()
{
return "SegmentTransactionalInsertAction{" +
- "segments=" + segments +
+ "segmentsToBeOverwritten=" + segmentsToBeOverwritten +
+ ", segments=" + segments +
", startMetadata=" + startMetadata +
", endMetadata=" + endMetadata +
'}';
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java
index 206fd5f..e268188 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java
@@ -26,8 +26,10 @@ import org.apache.druid.indexing.common.task.Task;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
- @JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class),
- @JsonSubTypes.Type(name = "lockTryAcquire", value = LockTryAcquireAction.class),
+ @JsonSubTypes.Type(name = "lockAcquire", value = TimeChunkLockAcquireAction.class),
+ @JsonSubTypes.Type(name = "lockTryAcquire", value = TimeChunkLockTryAcquireAction.class),
+ @JsonSubTypes.Type(name = "segmentLockTryAcquire", value = SegmentLockTryAcquireAction.class),
+ @JsonSubTypes.Type(name = "segmentLockAcquire", value = SegmentLockAcquireAction.class),
@JsonSubTypes.Type(name = "lockList", value = LockListAction.class),
@JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class),
@JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionPreconditions.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionPreconditions.java
deleted file mode 100644
index 6db8d8e..0000000
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionPreconditions.java
+++ /dev/null
@@ -1,94 +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.actions;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.druid.indexing.common.TaskLock;
-import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.indexing.overlord.TaskLockbox;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.timeline.DataSegment;
-import org.joda.time.DateTime;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-public class TaskActionPreconditions
-{
- static void checkLockCoversSegments(
- final Task task,
- final TaskLockbox taskLockbox,
- final Collection<DataSegment> segments
- )
- {
- if (!isLockCoversSegments(task, taskLockbox, segments)) {
- throw new ISE(
- "Segments[%s] are not covered by locks[%s] for task[%s]",
- segments,
- taskLockbox.findLocksForTask(task),
- task.getId()
- );
- }
- }
-
- @VisibleForTesting
- static boolean isLockCoversSegments(
- final Task task,
- final TaskLockbox taskLockbox,
- final Collection<DataSegment> segments
- )
- {
- // Verify that each of these segments falls under some lock
-
- // NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check
- // NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we
- // NOTE: insert some segments from the task but not others.
-
- final NavigableMap<DateTime, TaskLock> taskLockMap = getTaskLockMap(taskLockbox, task);
- if (taskLockMap.isEmpty()) {
- return false;
- }
-
- return segments.stream().allMatch(
- segment -> {
- final Entry<DateTime, TaskLock> entry = taskLockMap.floorEntry(segment.getInterval().getStart());
- if (entry == null) {
- return false;
- }
-
- final TaskLock taskLock = entry.getValue();
- return taskLock.getInterval().contains(segment.getInterval()) &&
- taskLock.getDataSource().equals(segment.getDataSource()) &&
- taskLock.getVersion().compareTo(segment.getVersion()) >= 0;
- }
- );
- }
-
- private static NavigableMap<DateTime, TaskLock> getTaskLockMap(TaskLockbox taskLockbox, Task task)
- {
- final List<TaskLock> taskLocks = taskLockbox.findLocksForTask(task);
- final NavigableMap<DateTime, TaskLock> taskLockMap = new TreeMap<>();
- taskLocks.forEach(taskLock -> taskLockMap.put(taskLock.getInterval().getStart(), taskLock));
- return taskLockMap;
- }
-}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java
new file mode 100644
index 0000000..a7feb4f
--- /dev/null
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.common.actions;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.druid.indexing.common.LockGranularity;
+import org.apache.druid.indexing.common.SegmentLock;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TimeChunkLock;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.indexing.overlord.TaskLockbox;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.timeline.DataSegment;
+import org.joda.time.DateTime;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+public class TaskLocks
+{
+ static void checkLockCoversSegments(
+ final Task task,
+ final TaskLockbox taskLockbox,
+ final Collection<DataSegment> segments
+ )
+ {
+ if (!isLockCoversSegments(task, taskLockbox, segments)) {
+ throw new ISE(
+ "Segments[%s] are not covered by locks[%s] for task[%s]",
+ segments,
+ taskLockbox.findLocksForTask(task),
+ task.getId()
+ );
+ }
+ }
+
+ @VisibleForTesting
+ static boolean isLockCoversSegments(
+ final Task task,
+ final TaskLockbox taskLockbox,
+ final Collection<DataSegment> segments
+ )
+ {
+ // Verify that each of these segments falls under some lock
+
+ // NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check
+ // NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we
+ // NOTE: insert some segments from the task but not others.
+
+ final NavigableMap<DateTime, List<TaskLock>> taskLockMap = getTaskLockMap(taskLockbox, task);
+ if (taskLockMap.isEmpty()) {
+ return false;
+ }
+
+ return isLockCoversSegments(taskLockMap, segments);
+ }
+
+ public static boolean isLockCoversSegments(
+ NavigableMap<DateTime, List<TaskLock>> taskLockMap,
+ Collection<DataSegment> segments
+ )
+ {
+ return segments.stream().allMatch(
+ segment -> {
+ final Entry<DateTime, List<TaskLock>> entry = taskLockMap.floorEntry(segment.getInterval().getStart());
+ if (entry == null) {
+ return false;
+ }
+
+ final List<TaskLock> locks = entry.getValue();
+ return locks.stream().anyMatch(
+ lock -> {
+ if (lock.getGranularity() == LockGranularity.TIME_CHUNK) {
+ final TimeChunkLock timeChunkLock = (TimeChunkLock) lock;
+ return timeChunkLock.getInterval().contains(segment.getInterval())
+ && timeChunkLock.getDataSource().equals(segment.getDataSource())
+ && timeChunkLock.getVersion().compareTo(segment.getVersion()) >= 0;
+ } else {
+ final SegmentLock segmentLock = (SegmentLock) lock;
+ return segmentLock.getInterval().contains(segment.getInterval())
+ && segmentLock.getDataSource().equals(segment.getDataSource())
+ && segmentLock.getVersion().compareTo(segment.getVersion()) >= 0
+ && segmentLock.getPartitionId() == segment.getShardSpec().getPartitionNum();
+ }
+ }
+ );
+ }
+ );
+ }
+
+ public static List<TaskLock> findLocksForSegments(
+ final Task task,
+ final TaskLockbox taskLockbox,
+ final Collection<DataSegment> segments
+ )
+ {
+ final NavigableMap<DateTime, List<TaskLock>> taskLockMap = getTaskLockMap(taskLockbox, task);
+ if (taskLockMap.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ final List<TaskLock> found = new ArrayList<>();
+ segments.forEach(segment -> {
+ final Entry<DateTime, List<TaskLock>> entry = taskLockMap.floorEntry(segment.getInterval().getStart());
+ if (entry == null) {
+ throw new ISE("Can't find lock for the interval of segment[%s]", segment.getId());
+ }
+
+ final List<TaskLock> locks = entry.getValue();
+ locks.forEach(lock -> {
+ if (lock.getGranularity() == LockGranularity.TIME_CHUNK) {
+ final TimeChunkLock timeChunkLock = (TimeChunkLock) lock;
+ if (timeChunkLock.getInterval().contains(segment.getInterval())
+ && timeChunkLock.getDataSource().equals(segment.getDataSource())
+ && timeChunkLock.getVersion().compareTo(segment.getVersion()) >= 0) {
+ found.add(lock);
+ }
+ } else {
+ final SegmentLock segmentLock = (SegmentLock) lock;
+ if (segmentLock.getInterval().contains(segment.getInterval())
+ && segmentLock.getDataSource().equals(segment.getDataSource())
+ && segmentLock.getVersion().compareTo(segment.getVersion()) >= 0
+ && segmentLock.getPartitionId() == segment.getShardSpec().getPartitionNum()) {
+ found.add(lock);
+ }
+ }
+ });
+ });
+ return found;
+ }
+
+ private static NavigableMap<DateTime, List<TaskLock>> getTaskLockMap(TaskLockbox taskLockbox, Task task)
+ {
+ final List<TaskLock> taskLocks = taskLockbox.findLocksForTask(task);
+ final NavigableMap<DateTime, List<TaskLock>> taskLockMap = new TreeMap<>();
+ taskLocks.forEach(taskLock -> taskLockMap.computeIfAbsent(taskLock.getInterval().getStart(), k -> new ArrayList<>())
+ .add(taskLock));
+ return taskLockMap;
+ }
+}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java
similarity index 75%
rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java
rename to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java
index 0b2e49d..83eec13 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java
@@ -28,11 +28,17 @@ import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.LockResult;
+import org.apache.druid.indexing.overlord.TimeChunkLockRequest;
import org.joda.time.Interval;
import javax.annotation.Nullable;
-public class LockAcquireAction implements TaskAction<TaskLock>
+/**
+ * TaskAction to acquire a {@link org.apache.druid.indexing.common.TimeChunkLock}.
+ * This action is a blocking operation and the caller could wait until it gets {@link TaskLock}
+ * (up to timeoutMs if it's > 0). It returns null if it fails to get a lock within timeout.
+ */
+public class TimeChunkLockAcquireAction implements TaskAction<TaskLock>
{
private final TaskLockType type;
@@ -43,7 +49,7 @@ public class LockAcquireAction implements TaskAction<TaskLock>
private final long timeoutMs;
@JsonCreator
- public LockAcquireAction(
+ public TimeChunkLockAcquireAction(
@JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility
@JsonProperty("interval") Interval interval,
@JsonProperty("timeoutMs") long timeoutMs
@@ -84,9 +90,11 @@ public class LockAcquireAction implements TaskAction<TaskLock>
public TaskLock perform(Task task, TaskActionToolbox toolbox)
{
try {
- final LockResult result = timeoutMs == 0 ?
- toolbox.getTaskLockbox().lock(type, task, interval) :
- toolbox.getTaskLockbox().lock(type, task, interval, timeoutMs);
+ final LockResult result = timeoutMs == 0
+ ? toolbox.getTaskLockbox()
+ .lock(task, new TimeChunkLockRequest(type, task, interval, null))
+ : toolbox.getTaskLockbox()
+ .lock(task, new TimeChunkLockRequest(type, task, interval, null), timeoutMs);
return result.isOk() ? result.getTaskLock() : null;
}
catch (InterruptedException e) {
@@ -103,7 +111,7 @@ public class LockAcquireAction implements TaskAction<TaskLock>
@Override
public String toString()
{
- return "LockAcquireAction{" +
+ return "TimeChunkLockAcquireAction{" +
"lockType=" + type +
", interval=" + interval +
", timeoutMs=" + timeoutMs +
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java
similarity index 79%
rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockTryAcquireAction.java
rename to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java
index f24266b..7c2b7e0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/LockTryAcquireAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java
@@ -27,11 +27,16 @@ import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.LockResult;
+import org.apache.druid.indexing.overlord.TimeChunkLockRequest;
import org.joda.time.Interval;
import javax.annotation.Nullable;
-public class LockTryAcquireAction implements TaskAction<TaskLock>
+/**
+ * TaskAction to try to acquire a {@link org.apache.druid.indexing.common.TimeChunkLock}.
+ * This action returns null immediately if it fails to get a lock for the given interval.
+ */
+public class TimeChunkLockTryAcquireAction implements TaskAction<TaskLock>
{
@JsonIgnore
private final TaskLockType type;
@@ -40,7 +45,7 @@ public class LockTryAcquireAction implements TaskAction<TaskLock>
private final Interval interval;
@JsonCreator
- public LockTryAcquireAction(
+ public TimeChunkLockTryAcquireAction(
@JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility
@JsonProperty("interval") Interval interval
)
@@ -72,7 +77,10 @@ public class LockTryAcquireAction implements TaskAction<TaskLock>
@Override
public TaskLock perform(Task task, TaskActionToolbox toolbox)
{
- final LockResult result = toolbox.getTaskLockbox().tryLock(type, task, interval);
+ final LockResult result = toolbox.getTaskLockbox().tryLock(
+ task,
+ new TimeChunkLockRequest(type, task, interval, null)
+ );
return result.isOk() ? result.getTaskLock() : null;
}
@@ -85,8 +93,8 @@ public class LockTryAcquireAction implements TaskAction<TaskLock>
@Override
public String toString()
{
- return "LockTryAcquireAction{" +
- "lockType=" + type +
+ return "TimeChunkLockTryAcquireAction{" +
+ ", type=" + type +
", interval=" + interval +
'}';
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java
index f872042..7d3e9de 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java
@@ -108,7 +108,8 @@ public class YeOldePlumberSchool implements PlumberSchool
config.getMaxRowsInMemory(),
TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()),
config.isReportParseExceptions(),
- config.getDedupColumn()
+ config.getDedupColumn(),
+ null
);
// Temporary directory to hold spilled segments.
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java
new file mode 100644
index 0000000..b00c646
--- /dev/null
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java
@@ -0,0 +1,384 @@
+/*
+ * 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.Iterables;
+import org.apache.druid.data.input.FirehoseFactory;
+import org.apache.druid.indexing.common.LockGranularity;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.actions.SegmentListUsedAction;
+import org.apache.druid.indexing.common.actions.TaskActionClient;
+import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
+import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
+import org.apache.druid.indexing.firehose.WindowedSegmentId;
+import org.apache.druid.java.util.common.JodaUtils;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.GranularityType;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.segment.indexing.granularity.GranularitySpec;
+import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.TimelineObjectHolder;
+import org.apache.druid.timeline.VersionedIntervalTimeline;
+import org.apache.druid.timeline.partition.PartitionChunk;
+import org.joda.time.Interval;
+import org.joda.time.Period;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+/**
+ * Abstract class for batch tasks like {@link IndexTask}.
+ * Provides some methods such as {@link #determineSegmentGranularity}, {@link #findInputSegments},
+ * and {@link #determineLockGranularityandTryLock} for easily acquiring task locks.
+ */
+public abstract class AbstractBatchIndexTask extends AbstractTask
+{
+ private static final Logger log = new Logger(AbstractBatchIndexTask.class);
+
+ private final SegmentLockHelper segmentLockHelper;
+
+ /**
+ * State to indicate that this task will use segmentLock or timeChunkLock.
+ * This is automatically set when {@link #determineLockGranularityandTryLock} is called.
+ */
+ private boolean useSegmentLock;
+
+ protected AbstractBatchIndexTask(String id, String dataSource, Map<String, Object> context)
+ {
+ super(id, dataSource, context);
+ segmentLockHelper = new SegmentLockHelper();
+ }
+
+ protected AbstractBatchIndexTask(
+ String id,
+ @Nullable String groupId,
+ @Nullable TaskResource taskResource,
+ String dataSource,
+ @Nullable Map<String, Object> context
+ )
+ {
+ super(id, groupId, taskResource, dataSource, context);
+ segmentLockHelper = new SegmentLockHelper();
+ }
+
+ /**
+ * Return true if this task can overwrite existing segments.
+ */
+ public abstract boolean requireLockExistingSegments();
+
+ /**
+ * Find segments to lock in the given intervals.
+ * If this task is intend to overwrite only some segments in those intervals, this method should return only those
+ * segments instead of entire segments in those intervals.
+ */
+ public abstract List<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> intervals)
+ throws IOException;
+
+ /**
+ * Returns true if this task is in the perfect (guaranteed) rollup mode.
+ */
+ public abstract boolean isPerfectRollup();
+
+ /**
+ * Returns the segmentGranularity defined in the ingestion spec.
+ */
+ @Nullable
+ public abstract Granularity getSegmentGranularity();
+
+ public boolean isUseSegmentLock()
+ {
+ return useSegmentLock;
+ }
+
+ public SegmentLockHelper getSegmentLockHelper()
+ {
+ return segmentLockHelper;
+ }
+
+ /**
+ * Determine lockGranularity to use and try to acquire necessary locks.
+ * This method respects the value of 'forceTimeChunkLock' in task context.
+ * If it's set to false or missing, this method checks if this task can use segmentLock.
+ */
+ protected boolean determineLockGranularityAndTryLock(
+ TaskActionClient client,
+ GranularitySpec granularitySpec
+ ) throws IOException
+ {
+ final List<Interval> intervals = granularitySpec.bucketIntervals().isPresent()
+ ? new ArrayList<>(granularitySpec.bucketIntervals().get())
+ : Collections.emptyList();
+ return determineLockGranularityandTryLock(client, intervals);
+ }
+
+ boolean determineLockGranularityandTryLock(TaskActionClient client, List<Interval> intervals) throws IOException
+ {
+ final boolean forceTimeChunkLock = getContextValue(
+ Tasks.FORCE_TIME_CHUNK_LOCK_KEY,
+ Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK
+ );
+ // Respect task context value most.
+ if (forceTimeChunkLock) {
+ log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
+ useSegmentLock = false;
+ if (!intervals.isEmpty()) {
+ return tryTimeChunkLock(client, intervals);
+ } else {
+ return true;
+ }
+ } else {
+ if (!intervals.isEmpty()) {
+ final LockGranularityDetermineResult result = determineSegmentGranularity(client, intervals);
+ useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT;
+ return tryLockWithDetermineResult(client, result);
+ } else {
+ return true;
+ }
+ }
+ }
+
+ boolean determineLockGranularityandTryLockWithSegments(TaskActionClient client, List<DataSegment> segments)
+ throws IOException
+ {
+ final boolean forceTimeChunkLock = getContextValue(
+ Tasks.FORCE_TIME_CHUNK_LOCK_KEY,
+ Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK
+ );
+ if (forceTimeChunkLock) {
+ log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
+ useSegmentLock = false;
+ return tryTimeChunkLock(
+ client,
+ new ArrayList<>(segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()))
+ );
+ } else {
+ final LockGranularityDetermineResult result = determineSegmentGranularity(segments);
+ useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT;
+ return tryLockWithDetermineResult(client, result);
+ }
+ }
+
+ private LockGranularityDetermineResult determineSegmentGranularity(TaskActionClient client, List<Interval> intervals)
+ throws IOException
+ {
+ if (requireLockExistingSegments()) {
+ if (isPerfectRollup()) {
+ log.info("Using timeChunk lock for perfect rollup");
+ return new LockGranularityDetermineResult(LockGranularity.TIME_CHUNK, intervals, null);
+ } else if (!intervals.isEmpty()) {
+ // This method finds segments falling in all given intervals and then tries to lock those segments.
+ // Thus, there might be a race between calling findSegmentsToLock() and determineSegmentGranularity(),
+ // i.e., a new segment can be added to the interval or an existing segment might be removed.
+ // Removed segments should be fine because indexing tasks would do nothing with removed segments.
+ // However, tasks wouldn't know about new segments added after findSegmentsToLock() call, it may missing those
+ // segments. This is usually fine, but if you want to avoid this, you should use timeChunk lock instead.
+ return determineSegmentGranularity(findSegmentsToLock(client, intervals));
+ } else {
+ log.info("Using segment lock for empty intervals");
+ return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, Collections.emptyList());
+ }
+ } else {
+ log.info("Using segment lock since we don't have to lock existing segments");
+ return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, Collections.emptyList());
+ }
+ }
+
+ private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranularityDetermineResult result)
+ throws IOException
+ {
+ if (result.lockGranularity == LockGranularity.TIME_CHUNK) {
+ return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals"));
+ } else {
+ return segmentLockHelper.verifyAndLockExistingSegments(
+ client,
+ Preconditions.checkNotNull(result.segments, "segments")
+ );
+ }
+ }
+
+ private boolean tryTimeChunkLock(TaskActionClient client, List<Interval> intervals) throws IOException
+ {
+ // In this case, the intervals to lock must be aligned with segmentGranularity if it's defined
+ final Set<Interval> uniqueIntervals = new HashSet<>();
+ for (Interval interval : JodaUtils.condenseIntervals(intervals)) {
+ final Granularity segmentGranularity = getSegmentGranularity();
+ if (segmentGranularity == null) {
+ uniqueIntervals.add(interval);
+ } else {
+ Iterables.addAll(uniqueIntervals, segmentGranularity.getIterable(interval));
+ }
+ }
+
+ for (Interval interval : uniqueIntervals) {
+ final TaskLock lock = client.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval));
+ if (lock == null) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private LockGranularityDetermineResult determineSegmentGranularity(List<DataSegment> segments)
+ {
+ if (segments.isEmpty()) {
+ log.info("Using segment lock for empty segments");
+ // Set useSegmentLock even though we don't get any locks.
+ // Note that we should get any lock before data ingestion if we are supposed to use timChunk lock.
+ return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, Collections.emptyList());
+ }
+
+ if (requireLockExistingSegments()) {
+ final Granularity granularityFromSegments = findGranularityFromSegments(segments);
+ @Nullable
+ final Granularity segmentGranularityFromSpec = getSegmentGranularity();
+ final List<Interval> intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList());
+
+ if (granularityFromSegments == null
+ || segmentGranularityFromSpec != null
+ && (!granularityFromSegments.equals(segmentGranularityFromSpec)
+ || segments.stream().anyMatch(segment -> !segmentGranularityFromSpec.isAligned(segment.getInterval())))) {
+ // This case is one of the followings:
+ // 1) Segments have different granularities.
+ // 2) Segment granularity in ingestion spec is different from the one of existig segments.
+ // 3) Some existing segments are not aligned with the segment granularity in the ingestion spec.
+ log.info("Detected segmentGranularity change. Using timeChunk lock");
+ return new LockGranularityDetermineResult(LockGranularity.TIME_CHUNK, intervals, null);
+ } else {
+ // Use segment lock
+ // Create a timeline to find latest segments only
+ final VersionedIntervalTimeline<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(
+ segments
+ );
+
+ final List<DataSegment> segmentsToLock = timeline
+ .lookup(JodaUtils.umbrellaInterval(intervals))
+ .stream()
+ .map(TimelineObjectHolder::getObject)
+ .flatMap(partitionHolder -> StreamSupport.stream(partitionHolder.spliterator(), false))
+ .map(PartitionChunk::getObject)
+ .collect(Collectors.toList());
+ log.info("No segmentGranularity change detected and it's not perfect rollup. Using segment lock");
+ return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, segmentsToLock);
+ }
+ } else {
+ // Set useSegmentLock even though we don't get any locks.
+ // Note that we should get any lock before data ingestion if we are supposed to use timChunk lock.
+ log.info("Using segment lock since we don't have to lock existing segments");
+ return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, Collections.emptyList());
+ }
+ }
+
+ @Nullable
+ static Granularity findGranularityFromSegments(List<DataSegment> segments)
+ {
+ if (segments.isEmpty()) {
+ return null;
+ }
+ final Period firstSegmentPeriod = segments.get(0).getInterval().toPeriod();
+ final boolean allHasSameGranularity = segments
+ .stream()
+ .allMatch(segment -> firstSegmentPeriod.equals(segment.getInterval().toPeriod()));
+ if (allHasSameGranularity) {
+ return GranularityType.fromPeriod(firstSegmentPeriod).getDefaultGranularity();
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * 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
+ * filtered by intervalsToRead, so they need to be locked.
+ *
+ * However, firehoseFactory is not IngestSegmentFirehoseFactory, it means this task will overwrite some segments
+ * with data read from some input source outside of Druid. As a result, only the segments falling in intervalsToRead
+ * should be locked.
+ */
+ protected static List<DataSegment> findInputSegments(
+ String dataSource,
+ TaskActionClient actionClient,
+ List<Interval> intervalsToRead,
+ FirehoseFactory firehoseFactory
+ ) throws IOException
+ {
+ if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
+ // intervalsToRead is ignored here.
+ final List<WindowedSegmentId> inputSegments = ((IngestSegmentFirehoseFactory) firehoseFactory).getSegments();
+ if (inputSegments == null) {
+ final Interval inputInterval = Preconditions.checkNotNull(
+ ((IngestSegmentFirehoseFactory) firehoseFactory).getInterval(),
+ "input interval"
+ );
+
+ return actionClient.submit(
+ new SegmentListUsedAction(dataSource, null, Collections.singletonList(inputInterval))
+ );
+ } else {
+ final List<String> inputSegmentIds = inputSegments.stream()
+ .map(WindowedSegmentId::getSegmentId)
+ .collect(Collectors.toList());
+ final List<DataSegment> dataSegmentsInIntervals = actionClient.submit(
+ new SegmentListUsedAction(
+ dataSource,
+ null,
+ inputSegments.stream()
+ .flatMap(windowedSegmentId -> windowedSegmentId.getIntervals().stream())
+ .collect(Collectors.toSet())
+ )
+ );
+ return dataSegmentsInIntervals.stream()
+ .filter(segment -> inputSegmentIds.contains(segment.getId().toString()))
+ .collect(Collectors.toList());
+ }
+ } else {
+ return actionClient.submit(new SegmentListUsedAction(dataSource, null, intervalsToRead));
+ }
+ }
+
+ private static class LockGranularityDetermineResult
+ {
+ private final LockGranularity lockGranularity;
+ @Nullable
+ private final List<Interval> intervals; // null for segmentLock
+ @Nullable
+ private final List<DataSegment> segments; // null for timeChunkLock
+
+ private LockGranularityDetermineResult(
+ LockGranularity lockGranularity,
+ @Nullable List<Interval> intervals,
+ @Nullable List<DataSegment> segments
+ )
+ {
+ this.lockGranularity = lockGranularity;
+ this.intervals = intervals;
+ this.segments = segments;
+ }
+ }
+}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java
index 6959677..9582b98 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java
@@ -23,8 +23,8 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.druid.indexing.common.TaskLockType;
-import org.apache.druid.indexing.common.actions.LockTryAcquireAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
+import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
import org.joda.time.Interval;
import java.util.Map;
@@ -72,7 +72,7 @@ public abstract class AbstractFixedIntervalTask extends AbstractTask
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
- return taskActionClient.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null;
+ return taskActionClient.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null;
}
@JsonProperty
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java
index 27e585e..370b30e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java
@@ -188,7 +188,7 @@ public abstract class AbstractTask implements Task
return ID_JOINER.join(objects);
}
- static String joinId(Object...objects)
+ static String joinId(Object... objects)
{
return ID_JOINER.join(objects);
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
index 81f4797..2d5867b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
@@ -44,12 +44,16 @@ import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
+import org.apache.druid.indexing.common.LockGranularity;
+import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
import org.apache.druid.indexing.common.TaskReport;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
+import org.apache.druid.indexing.common.actions.SegmentLockAcquireAction;
import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
+import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec;
import org.apache.druid.indexing.common.index.RealtimeAppenderatorTuningConfig;
@@ -84,6 +88,7 @@ import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory;
import org.apache.druid.segment.realtime.plumber.Committers;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.timeline.partition.NumberedShardSpecFactory;
import org.apache.druid.utils.CircularBuffer;
import javax.servlet.http.HttpServletRequest;
@@ -94,6 +99,7 @@ import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.io.File;
+import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -161,6 +167,9 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
private final AuthorizerMapper authorizerMapper;
@JsonIgnore
+ private final LockGranularity lockGranularity;
+
+ @JsonIgnore
private IngestionState ingestionState;
@JsonIgnore
@@ -195,6 +204,9 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
this.ingestionState = IngestionState.NOT_STARTED;
this.rowIngestionMeters = rowIngestionMetersFactory.createRowIngestionMeters();
+ this.lockGranularity = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK)
+ ? LockGranularity.TIME_CHUNK
+ : LockGranularity.SEGMENT;
}
@Override
@@ -274,7 +286,34 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
toolbox.getDataSegmentServerAnnouncer().announce();
toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode);
- driver.startJob();
+ driver.startJob(
+ segmentId -> {
+ try {
+ if (lockGranularity == LockGranularity.SEGMENT) {
+ return toolbox.getTaskActionClient().submit(
+ new SegmentLockAcquireAction(
+ TaskLockType.EXCLUSIVE,
+ segmentId.getInterval(),
+ segmentId.getVersion(),
+ segmentId.getShardSpec().getPartitionNum(),
+ 1000L
+ )
+ ).isOk();
+ } else {
+ return toolbox.getTaskActionClient().submit(
+ new TimeChunkLockAcquireAction(
+ TaskLockType.EXCLUSIVE,
+ segmentId.getInterval(),
+ 1000L
+ )
+ ) != null;
+ }
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ );
// Set up metrics emission
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
@@ -289,8 +328,15 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
int sequenceNumber = 0;
String sequenceName = makeSequenceName(getId(), sequenceNumber);
- final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> {
- final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments);
+ final TransactionalSegmentPublisher publisher = (mustBeNullOrEmptySegments, segments, commitMetadata) -> {
+ if (mustBeNullOrEmptySegments != null && !mustBeNullOrEmptySegments.isEmpty()) {
+ throw new ISE("WTH? stream ingestion tasks are overwriting segments[%s]", mustBeNullOrEmptySegments);
+ }
+ final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.appendAction(
+ segments,
+ null,
+ null
+ );
return toolbox.getTaskActionClient().submit(action);
};
@@ -726,7 +772,9 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
schema.getGranularitySpec().getSegmentGranularity(),
sequenceName,
previousSegmentId,
- skipSegmentLineageCheck
+ skipSegmentLineageCheck,
+ NumberedShardSpecFactory.instance(),
+ LockGranularity.TIME_CHUNK
)
),
toolbox.getSegmentHandoffNotifierFactory(),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java
new file mode 100644
index 0000000..8d5680e
--- /dev/null
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.indexing.common.task;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.TaskToolbox;
+import org.apache.druid.indexing.common.actions.LockListAction;
+import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
+import org.apache.druid.timeline.partition.ShardSpec;
+import org.apache.druid.timeline.partition.ShardSpecFactory;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * Allocates all necessary segments locally at the beginning and reuse them.
+ */
+class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator
+{
+ private final TaskToolbox toolbox;
+ private final String taskId;
+ private final String dataSource;
+ private final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec;
+ @Nullable
+ private final ShardSpecs shardSpecs;
+
+ // sequenceName -> segmentId
+ private final Map<String, SegmentIdWithShardSpec> sequenceNameToSegmentId;
+
+ CachingLocalSegmentAllocator(
+ TaskToolbox toolbox,
+ String taskId,
+ String dataSource,
+ Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec
+ ) throws IOException
+ {
+ this.toolbox = toolbox;
+ this.taskId = taskId;
+ this.dataSource = dataSource;
+ this.allocateSpec = allocateSpec;
+ this.sequenceNameToSegmentId = new HashMap<>();
+
+ final Map<Interval, List<SegmentIdWithShardSpec>> intervalToIds = getIntervalToSegmentIds();
+ final Map<Interval, List<ShardSpec>> shardSpecMap = new HashMap<>();
+
+ for (Map.Entry<Interval, List<SegmentIdWithShardSpec>> entry : intervalToIds.entrySet()) {
+ final Interval interval = entry.getKey();
+ final List<SegmentIdWithShardSpec> idsPerInterval = intervalToIds.get(interval);
+
+ for (SegmentIdWithShardSpec segmentIdentifier : idsPerInterval) {
+ shardSpecMap.computeIfAbsent(interval, k -> new ArrayList<>()).add(segmentIdentifier.getShardSpec());
+ // The shardSpecs for partitioning and publishing can be different if isExtendableShardSpecs = true.
+ sequenceNameToSegmentId.put(getSequenceName(interval, segmentIdentifier.getShardSpec()), segmentIdentifier);
+ }
+ }
+ shardSpecs = new ShardSpecs(shardSpecMap);
+ }
+
+ private Map<Interval, List<SegmentIdWithShardSpec>> getIntervalToSegmentIds() throws IOException
+ {
+ final Map<Interval, String> intervalToVersion = getToolbox().getTaskActionClient()
+ .submit(new LockListAction())
+ .stream()
+ .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
+ final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = getAllocateSpec();
+ final Map<Interval, List<SegmentIdWithShardSpec>> intervalToSegmentIds = new HashMap<>(allocateSpec.size());
+ for (Entry<Interval, Pair<ShardSpecFactory, Integer>> entry : allocateSpec.entrySet()) {
+ final Interval interval = entry.getKey();
+ final ShardSpecFactory shardSpecFactory = entry.getValue().lhs;
+ final int numSegmentsToAllocate = Preconditions.checkNotNull(
+ entry.getValue().rhs,
+ "numSegmentsToAllocate for interval[%s]",
+ interval
+ );
+
+ intervalToSegmentIds.put(
+ interval,
+ IntStream.range(0, numSegmentsToAllocate)
+ .mapToObj(i -> new SegmentIdWithShardSpec(
+ getDataSource(),
+ interval,
+ findVersion(intervalToVersion, interval),
+ shardSpecFactory.create(getToolbox().getObjectMapper(), i)
+ ))
+ .collect(Collectors.toList())
+ );
+ }
+ return intervalToSegmentIds;
+ }
+
+ private static String findVersion(Map<Interval, String> intervalToVersion, Interval interval)
+ {
+ return intervalToVersion.entrySet().stream()
+ .filter(entry -> entry.getKey().contains(interval))
+ .map(Entry::getValue)
+ .findFirst()
+ .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
+ }
+
+
+ TaskToolbox getToolbox()
+ {
+ return toolbox;
+ }
+
+ String getTaskId()
+ {
+ return taskId;
+ }
+
+ String getDataSource()
+ {
+ return dataSource;
+ }
+
+ Map<Interval, Pair<ShardSpecFactory, Integer>> getAllocateSpec()
+ {
+ return allocateSpec;
+ }
+
+ @Override
+ public SegmentIdWithShardSpec allocate(
+ InputRow row,
+ String sequenceName,
+ String previousSegmentId,
+ boolean skipSegmentLineageCheck
+ )
+ {
+ return sequenceNameToSegmentId.get(sequenceName);
+ }
+
+ @Override
+ public String getSequenceName(Interval interval, InputRow inputRow)
+ {
+ // Sequence name is based solely on the shardSpec, and there will only be one segment per sequence.
+ return getSequenceName(interval, shardSpecs.getShardSpec(interval, inputRow));
+ }
+
+ /**
+ * 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)
+ {
+ return StringUtils.format("%s_%s_%d", taskId, interval, shardSpec.getPartitionNum());
+ }
+}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
index 6a0ae54..5d3bdf8 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
@@ -96,14 +96,12 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
-import java.util.SortedSet;
import java.util.TreeMap;
-import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.StreamSupport;
-public class CompactionTask extends AbstractTask
+public class CompactionTask extends AbstractBatchIndexTask
{
private static final Logger log = new Logger(CompactionTask.class);
private static final String TYPE = "compact";
@@ -224,6 +222,7 @@ public class CompactionTask extends AbstractTask
@JsonProperty
@Nullable
+ @Override
public Granularity getSegmentGranularity()
{
return segmentGranularity;
@@ -255,25 +254,37 @@ public class CompactionTask extends AbstractTask
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_MERGE_TASK_PRIORITY);
}
- @VisibleForTesting
- SegmentProvider getSegmentProvider()
+ @Override
+ public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
- return segmentProvider;
+ final List<DataSegment> segments = segmentProvider.checkAndGetSegments(taskActionClient);
+ return determineLockGranularityandTryLockWithSegments(taskActionClient, segments);
}
@Override
- public boolean isReady(TaskActionClient taskActionClient) throws Exception
+ public boolean requireLockExistingSegments()
+ {
+ return true;
+ }
+
+ @Override
+ public List<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> intervals)
+ throws IOException
+ {
+ return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals));
+ }
+
+ @Override
+ public boolean isPerfectRollup()
{
- final SortedSet<Interval> intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
- intervals.add(segmentProvider.interval);
- return IndexTask.isReady(taskActionClient, intervals);
+ return tuningConfig != null && tuningConfig.isForceGuaranteedRollup();
}
@Override
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
if (indexTaskSpecs == null) {
- indexTaskSpecs = createIngestionSchema(
+ final List<IndexIngestionSpec> ingestionSpecs = createIngestionSchema(
toolbox,
segmentProvider,
partitionConfigurationManager,
@@ -284,19 +295,21 @@ public class CompactionTask extends AbstractTask
coordinatorClient,
segmentLoaderFactory,
retryPolicyFactory
- ).stream()
- .map(spec -> new IndexTask(
- getId(),
- getGroupId(),
- getTaskResource(),
- getDataSource(),
- spec,
- getContext(),
- authorizerMapper,
- chatHandlerProvider,
- rowIngestionMetersFactory
- ))
- .collect(Collectors.toList());
+ );
+ indexTaskSpecs = IntStream
+ .range(0, ingestionSpecs.size())
+ .mapToObj(i -> new IndexTask(
+ createIndexTaskSpecId(i),
+ getGroupId(),
+ getTaskResource(),
+ getDataSource(),
+ ingestionSpecs.get(i),
+ getContext(),
+ authorizerMapper,
+ chatHandlerProvider,
+ rowIngestionMetersFactory
+ ))
+ .collect(Collectors.toList());
}
if (indexTaskSpecs.isEmpty()) {
@@ -312,10 +325,15 @@ public class CompactionTask extends AbstractTask
log.info("Running indexSpec: " + json);
try {
- final TaskStatus eachResult = eachSpec.run(toolbox);
- if (!eachResult.isSuccess()) {
+ if (eachSpec.isReady(toolbox.getTaskActionClient())) {
+ final TaskStatus eachResult = eachSpec.run(toolbox);
+ if (!eachResult.isSuccess()) {
+ failCnt++;
+ log.warn("Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json);
+ }
+ } else {
failCnt++;
- log.warn("Failed to run indexSpec: [%s].\nTrying the next indexSpec.", json);
+ log.warn("indexSpec is not ready: [%s].\nTrying the next indexSpec.", json);
}
}
catch (Exception e) {
@@ -329,6 +347,11 @@ public class CompactionTask extends AbstractTask
}
}
+ private String createIndexTaskSpecId(int i)
+ {
+ return StringUtils.format("%s_%d", getId(), i);
+ }
+
/**
* Generate {@link IndexIngestionSpec} from input segments.
*
@@ -360,6 +383,7 @@ public class CompactionTask extends AbstractTask
}
// find metadata for interval
+ // queryableIndexAndSegments is sorted by the interval of the dataSegment
final List<Pair<QueryableIndex, DataSegment>> queryableIndexAndSegments = loadSegments(
timelineSegments,
segmentFileMap,
@@ -387,7 +411,6 @@ public class CompactionTask extends AbstractTask
final List<Pair<QueryableIndex, DataSegment>> segmentsToCompact = entry.getValue();
final DataSchema dataSchema = createDataSchema(
segmentProvider.dataSource,
- interval,
segmentsToCompact,
dimensionsSpec,
metricsSpec,
@@ -416,7 +439,6 @@ public class CompactionTask extends AbstractTask
// given segment granularity
final DataSchema dataSchema = createDataSchema(
segmentProvider.dataSource,
- segmentProvider.interval,
queryableIndexAndSegments,
dimensionsSpec,
metricsSpec,
@@ -474,7 +496,7 @@ public class CompactionTask extends AbstractTask
SegmentProvider segmentProvider
) throws IOException, SegmentLoadingException
{
- final List<DataSegment> usedSegments = segmentProvider.checkAndGetSegments(toolbox);
+ final List<DataSegment> usedSegments = segmentProvider.checkAndGetSegments(toolbox.getTaskActionClient());
final Map<DataSegment, File> segmentFileMap = toolbox.fetchSegments(usedSegments);
final List<TimelineObjectHolder<String, DataSegment>> timelineSegments = VersionedIntervalTimeline
.forSegments(usedSegments)
@@ -484,7 +506,6 @@ public class CompactionTask extends AbstractTask
private static DataSchema createDataSchema(
String dataSource,
- Interval totalInterval,
List<Pair<QueryableIndex, DataSegment>> queryableIndexAndSegments,
@Nullable DimensionsSpec dimensionsSpec,
@Nullable AggregatorFactory[] metricsSpec,
@@ -508,6 +529,10 @@ public class CompactionTask extends AbstractTask
return isRollup != null && isRollup;
});
+ final Interval totalInterval = JodaUtils.umbrellaInterval(
+ queryableIndexAndSegments.stream().map(p -> p.rhs.getInterval()).collect(Collectors.toList())
+ );
+
final GranularitySpec granularitySpec = new UniformGranularitySpec(
Preconditions.checkNotNull(segmentGranularity),
Granularities.NONE,
@@ -693,6 +718,7 @@ public class CompactionTask extends AbstractTask
{
private final String dataSource;
private final Interval interval;
+ @Nullable
private final List<DataSegment> segments;
SegmentProvider(String dataSource, Interval interval)
@@ -710,21 +736,22 @@ public class CompactionTask extends AbstractTask
segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)),
"segments should have the same dataSource"
);
- this.segments = segments;
this.dataSource = dataSource;
+ this.segments = segments;
this.interval = JodaUtils.umbrellaInterval(
segments.stream().map(DataSegment::getInterval).collect(Collectors.toList())
);
}
+ @Nullable
List<DataSegment> getSegments()
{
return segments;
}
- List<DataSegment> checkAndGetSegments(TaskToolbox toolbox) throws IOException
+ List<DataSegment> checkAndGetSegments(TaskActionClient actionClient) throws IOException
{
- final List<DataSegment> usedSegments = toolbox.getTaskActionClient().submit(
+ final List<DataSegment> usedSegments = actionClient.submit(
new SegmentListUsedAction(dataSource, interval, null)
);
final TimelineLookup<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(usedSegments);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
index 2571c57..f8728c0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
@@ -45,16 +45,19 @@ import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.TaskReport;
import org.apache.druid.indexing.common.TaskToolbox;
-import org.apache.druid.indexing.common.actions.LockAcquireAction;
-import org.apache.druid.indexing.common.actions.LockTryAcquireAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
+import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction;
+import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.stats.RowIngestionMeters;
import org.apache.druid.indexing.hadoop.OverlordActionBasedUsedSegmentLister;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
+import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.server.security.Action;
@@ -64,6 +67,7 @@ import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.util.ToolRunner;
import org.joda.time.Interval;
+import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
@@ -197,12 +201,42 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
intervals.get()
)
);
- return taskActionClient.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null;
+ return taskActionClient.submit(new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null;
} else {
return true;
}
}
+ @Override
+ public boolean requireLockExistingSegments()
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public List<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> intervals)
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean isPerfectRollup()
+ {
+ return true;
+ }
+
+ @Nullable
+ @Override
+ public Granularity getSegmentGranularity()
+ {
+ final GranularitySpec granularitySpec = spec.getDataSchema().getGranularitySpec();
+ if (granularitySpec instanceof ArbitraryGranularitySpec) {
+ return null;
+ } else {
+ return granularitySpec.getSegmentGranularity();
+ }
+ }
+
@JsonProperty("spec")
public HadoopIngestionSpec getSpec()
{
@@ -344,7 +378,7 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler
// Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, the below line can incur http timeout error.
final TaskLock lock = Preconditions.checkNotNull(
toolbox.getTaskActionClient().submit(
- new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs)
+ new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs)
),
"Cannot acquire a lock for interval[%s]", interval
);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java
index 768735d..f32ab28 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopTask.java
@@ -44,7 +44,7 @@ import java.util.List;
import java.util.Map;
-public abstract class HadoopTask extends AbstractTask
+public abstract class HadoopTask extends AbstractBatchIndexTask
{
private static final Logger log = new Logger(HadoopTask.class);
private static final ExtensionsConfig extensionsConfig;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
index 07a507a..b6a4126 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
@@ -28,7 +28,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
@@ -40,15 +39,13 @@ import org.apache.druid.data.input.Rows;
import org.apache.druid.hll.HyperLogLogCollector;
import org.apache.druid.indexer.IngestionState;
import org.apache.druid.indexer.TaskStatus;
-import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport;
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
-import org.apache.druid.indexing.common.TaskLock;
+import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
import org.apache.druid.indexing.common.TaskReport;
import org.apache.druid.indexing.common.TaskToolbox;
-import org.apache.druid.indexing.common.actions.SegmentAllocateAction;
import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.stats.RowIngestionMeters;
@@ -57,6 +54,7 @@ import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Comparators;
@@ -68,6 +66,7 @@ import org.apache.druid.segment.indexing.IOConfig;
import org.apache.druid.segment.indexing.IngestionSpec;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.indexing.TuningConfig;
+import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
@@ -78,7 +77,6 @@ import org.apache.druid.segment.realtime.appenderator.Appenderators;
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
-import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
@@ -88,12 +86,12 @@ import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
+import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.ShardSpec;
+import org.apache.druid.timeline.partition.ShardSpecFactory;
import org.apache.druid.utils.CircularBuffer;
-import org.apache.druid.utils.CollectionUtils;
import org.codehaus.plexus.util.FileUtils;
-import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.Period;
@@ -108,24 +106,20 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.io.File;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Map.Entry;
import java.util.Objects;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeMap;
-import java.util.TreeSet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-public class IndexTask extends AbstractTask implements ChatHandler
+public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
{
private static final Logger log = new Logger(IndexTask.class);
private static final HashFunction hashFunction = Hashing.murmur3_128();
@@ -160,22 +154,22 @@ public class IndexTask extends AbstractTask implements ChatHandler
private final Optional<ChatHandlerProvider> chatHandlerProvider;
@JsonIgnore
- private FireDepartmentMetrics buildSegmentsFireDepartmentMetrics;
+ private final RowIngestionMeters determinePartitionsMeters;
@JsonIgnore
- private CircularBuffer<Throwable> buildSegmentsSavedParseExceptions;
+ private final RowIngestionMeters buildSegmentsMeters;
@JsonIgnore
- private CircularBuffer<Throwable> determinePartitionsSavedParseExceptions;
+ private FireDepartmentMetrics buildSegmentsFireDepartmentMetrics;
@JsonIgnore
- private String errorMsg;
+ private CircularBuffer<Throwable> buildSegmentsSavedParseExceptions;
@JsonIgnore
- private final RowIngestionMeters determinePartitionsMeters;
+ private CircularBuffer<Throwable> determinePartitionsSavedParseExceptions;
@JsonIgnore
- private final RowIngestionMeters buildSegmentsMeters;
+ private String errorMsg;
@JsonCreator
public IndexTask(
@@ -252,36 +246,44 @@ public class IndexTask extends AbstractTask implements ChatHandler
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
- final Optional<SortedSet<Interval>> intervals = ingestionSchema.getDataSchema()
- .getGranularitySpec()
- .bucketIntervals();
+ return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.dataSchema.getGranularitySpec());
+ }
- if (intervals.isPresent()) {
- return isReady(taskActionClient, intervals.get());
- } else {
- return true;
- }
+ @Override
+ public boolean requireLockExistingSegments()
+ {
+ return isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)
+ || !ingestionSchema.ioConfig.isAppendToExisting();
}
- static boolean isReady(TaskActionClient actionClient, SortedSet<Interval> intervals) throws IOException
+ @Override
+ public List<DataSegment> findSegmentsToLock(TaskActionClient taskActionClient, List<Interval> intervals)
+ throws IOException
{
- // Sanity check preventing empty intervals (which cannot be locked, and don't make sense anyway).
- for (Interval interval : intervals) {
- if (interval.toDurationMillis() == 0) {
- throw new ISE("Cannot run with empty interval[%s]", interval);
- }
- }
+ return findInputSegments(
+ getDataSource(),
+ taskActionClient,
+ intervals,
+ ingestionSchema.ioConfig.firehoseFactory
+ );
+ }
- final List<TaskLock> locks = getTaskLocks(actionClient);
- if (locks.size() == 0) {
- try {
- Tasks.tryAcquireExclusiveLocks(actionClient, intervals);
- }
- catch (Exception e) {
- return false;
- }
+ @Override
+ public boolean isPerfectRollup()
+ {
+ return isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig);
+ }
+
+ @Nullable
+ @Override
+ public Granularity getSegmentGranularity()
+ {
+ final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec();
+ if (granularitySpec instanceof ArbitraryGranularitySpec) {
+ return null;
+ } else {
+ return granularitySpec.getSegmentGranularity();
}
- return true;
}
@GET
@@ -433,33 +435,32 @@ public class IndexTask extends AbstractTask implements ChatHandler
// Initialize maxRowsPerSegment and maxTotalRows lazily
final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig;
- @Nullable final Integer maxRowsPerSegment = getValidMaxRowsPerSegment(tuningConfig);
- @Nullable final Long maxTotalRows = getValidMaxTotalRows(tuningConfig);
- final ShardSpecs shardSpecs = determineShardSpecs(toolbox, firehoseFactory, firehoseTempDir, maxRowsPerSegment);
+ @Nullable
+ final Integer maxRowsPerSegment = getValidMaxRowsPerSegment(tuningConfig);
+ @Nullable
+ final Long maxTotalRows = getValidMaxTotalRows(tuningConfig);
+ // Spec for segment allocation. This is used only for perfect rollup mode.
+ // See createSegmentAllocator().
+ final Map<Interval, Pair<ShardSpecFactory, Integer>> allocateSpec = determineShardSpecs(
+ toolbox,
+ firehoseFactory,
+ firehoseTempDir,
+ maxRowsPerSegment
+ );
... 12215 lines suppressed ...
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org