From 17a1158234df24692289ca1679335b3405896f7c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 22 Apr 2019 15:51:14 -0700 Subject: [PATCH 01/52] Segment locking --- .../common/granularity/AllGranularity.java | 9 + .../granularity/DurationGranularity.java | 10 + .../util/common/granularity/Granularity.java | 5 + .../common/granularity/NoneGranularity.java | 10 + .../common/granularity/PeriodGranularity.java | 7 + .../apache/druid/timeline/DataSegment.java | 88 +- .../apache/druid/timeline/Overshadowable.java | 80 ++ .../org/apache/druid/timeline/SegmentId.java | 1 - .../apache/druid/timeline/TimelineLookup.java | 2 +- .../druid/timeline/TimelineObjectHolder.java | 25 +- .../timeline/VersionedIntervalTimeline.java | 125 ++- .../timeline/partition/AtomicUpdateGroup.java | 181 ++++ .../partition/HashBasedNumberedShardSpec.java | 14 +- .../HashBasedNumberedShardSpecFactory.java | 101 +++ .../partition/ImmutablePartitionHolder.java | 6 +- .../partition/IntegerPartitionChunk.java | 6 +- .../timeline/partition/LinearShardSpec.java | 6 + .../partition/LinearShardSpecFactory.java | 58 ++ .../timeline/partition/NoneShardSpec.java | 8 +- .../partition/NoneShardSpecFactory.java | 56 ++ .../NumberedOverwritingPartitionChunk.java | 115 +++ .../NumberedOverwritingShardSpec.java | 224 +++++ .../NumberedOverwritingShardSpecFactory.java | 88 ++ .../partition/NumberedPartitionChunk.java | 10 + .../timeline/partition/NumberedShardSpec.java | 6 + .../partition/NumberedShardSpecFactory.java | 61 ++ .../partition/OvershadowableManager.java | 552 ++++++++++++ .../partition/OverwritingPartitionChunk.java | 25 + .../partition/OverwritingShardSpec.java | 31 + .../timeline/partition/PartitionHolder.java | 81 +- .../druid/timeline/partition/ShardSpec.java | 32 +- .../timeline/partition/ShardSpecFactory.java | 42 + .../partition/SingleDimensionShardSpec.java | 17 +- .../SingleElementPartitionChunk.java | 2 +- .../partition/StringPartitionChunk.java | 11 + .../druid/timeline/DataSegmentTest.java | 11 +- .../VersionedIntervalTimelineTest.java | 825 +++++++++++++----- .../partition/IntegerPartitionChunkTest.java | 49 +- .../timeline/partition/NoneShardSpecTest.java | 7 + .../kafka/LegacyKafkaIndexTaskRunner.java | 21 +- .../indexing/kafka/KafkaIndexTaskTest.java | 381 ++++---- .../kinesis/KinesisIndexTaskTest.java | 344 ++++---- .../druid/indexer/IndexGeneratorJob.java | 1 + .../indexing/common/LockGranularity.java | 26 + .../druid/indexing/common/SegmentLock.java | 238 +++++ .../druid/indexing/common/TaskLock.java | 190 +--- .../druid/indexing/common/TaskLockType.java | 2 +- .../druid/indexing/common/TimeChunkLock.java | 220 +++++ .../common/actions/LocalTaskActionClient.java | 19 +- .../common/actions/SegmentAllocateAction.java | 85 +- .../actions/SegmentBulkAllocateAction.java | 145 +++ .../actions/SegmentLockAquireAction.java | 132 +++ .../actions/SegmentLockTryAcquireAction.java | 117 +++ .../actions/SegmentMetadataUpdateAction.java | 12 +- .../common/actions/SegmentNukeAction.java | 12 +- .../SegmentTransactionalInsertAction.java | 32 +- .../indexing/common/actions/TaskAction.java | 5 +- .../actions/TaskActionPreconditions.java | 44 +- ...n.java => TimeChunkLockAcquireAction.java} | 15 +- ...ava => TimeChunkLockTryAcquireAction.java} | 14 +- .../common/index/YeOldePlumberSchool.java | 3 +- .../task/AbstractFixedIntervalTask.java | 5 +- .../indexing/common/task/AbstractTask.java | 261 ++++++ .../AppenderatorDriverRealtimeIndexTask.java | 55 +- .../indexing/common/task/ArchiveTask.java | 29 + .../task/CachingLocalSegmentAllocator.java | 101 +++ .../task/CachingRemoteSegmentAllocator.java | 54 ++ .../common/task/CachingSegmentAllocator.java | 122 +++ .../indexing/common/task/CompactionTask.java | 109 ++- .../indexing/common/task/HadoopIndexTask.java | 36 +- .../druid/indexing/common/task/IndexTask.java | 291 +++--- .../task/IndexTaskSegmentAllocator.java | 28 + .../druid/indexing/common/task/KillTask.java | 82 +- .../common/task/LocalSegmentAllocator.java | 99 +++ .../druid/indexing/common/task/MoveTask.java | 29 + .../druid/indexing/common/task/NoopTask.java | 31 + .../common/task/RealtimeIndexTask.java | 40 +- .../common/task/RemoteSegmentAllocator.java | 106 +++ .../indexing/common/task/RestoreTask.java | 29 + .../druid/indexing/common/task/Tasks.java | 22 - .../batch/parallel/ParallelIndexSubTask.java | 185 +++- .../parallel/ParallelIndexSupervisorTask.java | 113 +-- .../indexing/overlord/CriticalAction.java | 2 +- .../druid/indexing/overlord/LockRequest.java | 46 + .../overlord/LockRequestForNewSegment.java | 183 ++++ .../druid/indexing/overlord/LockResult.java | 27 +- .../overlord/SpecificSegmentLockRequest.java | 172 ++++ .../druid/indexing/overlord/TaskLockbox.java | 634 +++++++------- .../overlord/TimeChunkLockRequest.java | 147 ++++ .../SeekableStreamIndexTask.java | 33 +- .../SeekableStreamIndexTaskRunner.java | 21 +- .../druid/indexing/common/TaskLockTest.java | 94 ++ .../druid/indexing/common/TestTasks.java | 57 ++ .../actions/RemoteTaskActionClientTest.java | 3 +- .../actions/SegmentAllocateActionTest.java | 405 +++++---- .../actions/SegmentInsertActionTest.java | 16 +- .../SegmentTransactionalInsertActionTest.java | 14 +- .../common/actions/SurrogateActionTest.java | 5 +- .../actions/TaskActionPreconditionsTest.java | 19 +- .../common/actions/TaskActionTestKit.java | 2 +- ...va => TimeChunkLockAcquireActionTest.java} | 14 +- ...=> TimeChunkLockTryAcquireActionTest.java} | 14 +- ...penderatorDriverRealtimeIndexTaskTest.java | 5 +- .../common/task/CompactionTaskRunTest.java | 313 ++++++- .../common/task/CompactionTaskTest.java | 4 +- .../indexing/common/task/HadoopTaskTest.java | 31 + .../indexing/common/task/IndexTaskTest.java | 332 +++---- .../common/task/IngestionTestBase.java | 298 +++++-- .../indexing/common/task/KillTaskTest.java | 107 +++ .../common/task/RealtimeIndexTaskTest.java | 2 +- ...stractParallelIndexSupervisorTaskTest.java | 23 +- ...rallelIndexSupervisorTaskResourceTest.java | 62 +- .../ParallelIndexSupervisorTaskTest.java | 33 +- .../IngestSegmentFirehoseFactoryTest.java | 47 +- .../indexing/overlord/RealtimeishTask.java | 34 +- .../SingleTaskBackgroundRunnerTest.java | 30 + .../indexing/overlord/TaskLifecycleTest.java | 99 ++- .../overlord/TaskLockBoxConcurrencyTest.java | 34 +- .../indexing/overlord/TaskLockboxTest.java | 565 ++++++++++-- .../overlord/http/OverlordResourceTest.java | 28 + ...TestIndexerMetadataStorageCoordinator.java | 12 +- .../druid/java/util/common/IntRange.java | 91 ++ .../druid/java/util/common/IntRanges.java | 32 + pom.xml | 2 +- .../segment/ReferenceCountingSegment.java | 75 +- .../druid/query/QueryRunnerTestHelper.java | 6 +- .../select/MultiSegmentSelectQueryTest.java | 10 +- .../TimeBoundaryQueryRunnerTest.java | 8 +- .../druid/segment/SchemalessIndexTest.java | 55 +- .../apache/druid/client/SegmentLoadInfo.java | 33 +- .../druid/client/selector/ServerSelector.java | 35 +- .../IndexerMetadataStorageCoordinator.java | 5 +- .../IndexerSQLMetadataStorageCoordinator.java | 141 +-- .../appenderator/AppenderatorImpl.java | 9 +- .../realtime/appenderator/Appenderators.java | 8 - .../appenderator/BaseAppenderatorDriver.java | 2 +- .../appenderator/BatchAppenderatorDriver.java | 14 +- .../appenderator/SegmentIdWithShardSpec.java | 15 + .../realtime/appenderator/SegmentLocker.java | 25 + .../StreamAppenderatorDriver.java | 10 +- .../TransactionalSegmentPublisher.java | 66 +- .../realtime/plumber/RealtimePlumber.java | 2 +- .../druid/segment/realtime/plumber/Sink.java | 35 +- .../apache/druid/server/SegmentManager.java | 15 +- .../DruidCoordinatorCleanupOvershadowed.java | 2 +- .../helper/DruidCoordinatorRuleRunner.java | 2 +- .../client/CachingClusteredClientTest.java | 83 +- ...exerSQLMetadataStorageCoordinatorTest.java | 184 +++- .../BatchAppenderatorDriverTest.java | 8 +- .../StreamAppenderatorDriverFailTest.java | 16 +- .../StreamAppenderatorDriverTest.java | 10 +- .../helper/NewestSegmentFirstPolicyTest.java | 2 +- .../server/shard/NumberedShardSpecTest.java | 125 ++- .../HashBasedNumberedShardSpecTest.java | 1 - .../sql/calcite/schema/SystemSchemaTest.java | 24 +- .../SpecificSegmentsQuerySegmentWalker.java | 15 +- 156 files changed, 9578 insertions(+), 2427 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/timeline/Overshadowable.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecFactory.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpecFactory.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpecFactory.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/OverwritingPartitionChunk.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentLock.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java rename indexing-service/src/main/java/org/apache/druid/indexing/common/actions/{LockAcquireAction.java => TimeChunkLockAcquireAction.java} (82%) rename indexing-service/src/main/java/org/apache/druid/indexing/common/actions/{LockTryAcquireAction.java => TimeChunkLockTryAcquireAction.java} (85%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/SpecificSegmentLockRequest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java rename indexing-service/src/test/java/org/apache/druid/indexing/common/actions/{LockAcquireActionTest.java => TimeChunkLockAcquireActionTest.java} (84%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/actions/{LockTryAcquireActionTest.java => TimeChunkLockTryAcquireActionTest.java} (82%) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java create mode 100644 java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java create mode 100644 java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentLocker.java 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 ce6a7966b3ac..25dcde897f8e 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 @@ -60,6 +60,15 @@ public DateTime toDate(String filePath, Formatter formatter) throw new UnsupportedOperationException("This method should not be invoked for this granularity type"); } + /** + * All granularity matches no interval. + */ + @Override + public boolean match(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 a48c071b5d22..78748836930b 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.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; @@ -94,6 +95,15 @@ public DateTime toDate(String filePath, Formatter formatter) throw new UnsupportedOperationException("This method should not be invoked for this granularity type"); } + @Override + public boolean match(Interval interval) + { + if (interval.toDurationMillis() == duration) { + return (interval.getStartMillis() - origin) % duration == 0; + } + return false; + } + @Override public byte[] getCacheKey() { 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 e2b7c33e5780..ad9ccd8a8b92 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 static List granularitiesFinerThan(final Granularity gran0) 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 match(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 f38241383c10..8e7ea6373502 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; /** @@ -57,6 +58,15 @@ public DateTime toDate(String filePath, Formatter formatter) throw new UnsupportedOperationException("This method should not be invoked for this granularity type"); } + /** + * None granularity matches to any interval. + */ + @Override + public boolean match(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 a88d53088ad1..13f5e12b7def 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.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; @@ -136,6 +137,12 @@ public DateTime toDate(String filePath, Formatter formatter) return null; } + @Override + public boolean match(Interval interval) + { + return bucket(interval.getStart()).equals(interval); + } + @Override public byte[] getCacheKey() { 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 97fd14a58315..fb4a876d18f4 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 @@ * {@link SegmentId} of the segment. */ @PublicApi -public class DataSegment implements Comparable +public class DataSegment implements Comparable, Overshadowable { /* * The difference between this class and org.apache.druid.segment.Segment is that this class contains the segment @@ -93,6 +93,29 @@ public static class PruneLoadSpecHolder private final ShardSpec shardSpec; private final long size; + public DataSegment( + SegmentId segmentId, + Map loadSpec, + List dimensions, + List 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, @@ -251,6 +274,50 @@ public SegmentId getId() return id; } + @Override + public boolean isOvershadow(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 new SegmentDescriptor(getInterval(), getVersion(), shardSpec.getPartitionNum()); @@ -271,6 +338,11 @@ public DataSegment withMetrics(List metrics) 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 +383,13 @@ public int hashCode() 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 000000000000..1a9790e12157 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java @@ -0,0 +1,80 @@ +/* + * 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 check the overshadowing relation between objeccts. + * 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 +{ + default boolean isOvershadow(T other) + { + return containsRootPartition(other) && getMinorVersion() > other.getMinorVersion(); + } + + 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(); + + default short getStartRootPartitionIdAsShort() + { + return (short) getStartRootPartitionId(); + } + + default short getEndRootPartitionIdAsShort() + { + return (short) getEndRootPartitionId(); + } + + // TODO: majorVersion? + + 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 2c21e21bc6e4..f938f4ded0ed 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentId.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentId.java @@ -322,7 +322,6 @@ public Interval getInterval() 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 af7544083756..45ccadde81ed 100644 --- a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java +++ b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java @@ -25,7 +25,7 @@ import java.util.List; -public interface TimelineLookup +public interface TimelineLookup> { /** 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 3feca88495c3..4aa1738b2aca 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 org.apache.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; +import java.util.Objects; + /** */ -public class TimelineObjectHolder implements LogicalSegment +public class TimelineObjectHolder> implements LogicalSegment { private final Interval interval; private final Interval trueInterval; @@ -73,6 +75,27 @@ public PartitionHolder getObject() return object; } + @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() { 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 36c177dfb0f0..0762fa78c78d 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -40,6 +40,7 @@ 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; @@ -59,21 +60,27 @@ * 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 implements TimelineLookup +public class VersionedIntervalTimeline> implements TimelineLookup { private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); - final NavigableMap completePartitionsTimeline = new TreeMap( + // Below timelines stores only *visible* timelineEntries + // adjusted interval -> timelineEntry + private final NavigableMap completePartitionsTimeline = new TreeMap<>( Comparators.intervalsByStartThenEnd() ); - final NavigableMap incompletePartitionsTimeline = new TreeMap( + // IncompletePartitionsTimeline also includes completePartitionsTimeline + // adjusted interval -> timelineEntry + @VisibleForTesting + final NavigableMap incompletePartitionsTimeline = new TreeMap<>( Comparators.intervalsByStartThenEnd() ); + // true interval -> version -> timelineEntry private final Map> allTimelineEntries = new HashMap<>(); private final Comparator versionComparator; @@ -92,7 +99,9 @@ public static VersionedIntervalTimeline forSegments(Iterabl public static VersionedIntervalTimeline forSegments(Iterator segments) { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( + Ordering.natural() + ); addSegments(timeline, segments); return timeline; } @@ -109,7 +118,6 @@ public static void addSegments( ); } - @VisibleForTesting public Map> getAllTimelineEntries() { return allTimelineEntries; @@ -159,7 +167,7 @@ private void addAll( } // "isComplete" is O(objects in holder) so defer it to the end of addAll. - for (Map.Entry entry : allEntries.entrySet()) { + for (Entry entry : allEntries.entrySet()) { Interval interval = entry.getValue(); if (entry.getKey().getPartitionHolder().isComplete()) { @@ -213,7 +221,7 @@ public PartitionHolder findEntry(Interval interval, VersionType vers { try { lock.readLock().lock(); - for (Map.Entry> entry : allTimelineEntries.entrySet()) { + for (Entry> entry : allTimelineEntries.entrySet()) { if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) { TimelineEntry foundEntry = entry.getValue().get(version); if (foundEntry != null) { @@ -307,20 +315,18 @@ private TimelineObjectHolder timelineEntryToObjectHolde ); } - public Set> findOvershadowed() + public Set> findFullyOvershadowed() { + lock.readLock().lock(); try { - lock.readLock().lock(); - Set> retVal = new HashSet<>(); - - Map> overShadowed = new HashMap<>(); + // 1. Put all timelineEntries and remove all visible entries to find out only non-visible timelineEntries. + final Map> overShadowed = new HashMap<>(); for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { - Map versionCopy = new HashMap<>(); - versionCopy.putAll(versionEntry.getValue()); + Map versionCopy = new HashMap<>(versionEntry.getValue()); overShadowed.put(versionEntry.getKey(), versionCopy); } - for (Map.Entry entry : completePartitionsTimeline.entrySet()) { + for (Entry entry : completePartitionsTimeline.entrySet()) { Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval()); if (versionEntry != null) { versionEntry.remove(entry.getValue().getVersion()); @@ -330,7 +336,7 @@ public Set> findOvershadowed() } } - for (Map.Entry entry : incompletePartitionsTimeline.entrySet()) { + for (Entry entry : incompletePartitionsTimeline.entrySet()) { Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval()); if (versionEntry != null) { versionEntry.remove(entry.getValue().getVersion()); @@ -340,10 +346,24 @@ public Set> findOvershadowed() } } - for (Map.Entry> versionEntry : overShadowed.entrySet()) { - for (Map.Entry entry : versionEntry.getValue().entrySet()) { - TimelineEntry object = entry.getValue(); - retVal.add(timelineEntryToObjectHolder(object)); + final Set> retVal = new HashSet<>(); + for (Entry> versionEntry : overShadowed.entrySet()) { + for (Entry 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()) { + if (!entry.partitionHolder.getOvershadowed().isEmpty()) { + retVal.add( + new TimelineObjectHolder<>( + entry.trueInterval, + entry.version, + new PartitionHolder<>(new ArrayList<>(entry.partitionHolder.getOvershadowed())) + ) + ); } } @@ -354,14 +374,18 @@ public Set> findOvershadowed() } } - public boolean isOvershadowed(Interval interval, VersionType version) + // TODO: check this is correct + // TODO: add test + public boolean isOvershadowed(Interval interval, VersionType version, ObjectType object) { 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()); + return majorVersionCompare < 0 + || (majorVersionCompare == 0 && entry.partitionHolder.stream().anyMatch(chunk -> chunk.getObject().isOvershadow(object))); } Interval lower = completePartitionsTimeline.floorKey( @@ -377,13 +401,23 @@ public boolean isOvershadowed(Interval interval, VersionType version) 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().isOvershadow(object))) { + return false; + } + } + prev = curr; curr = completePartitionsTimeline.higherKey(curr); @@ -453,21 +487,32 @@ private boolean addAtKey( } 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())) { + // | cur | + // | entry | + // => |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 { @@ -475,17 +520,27 @@ private boolean addAtKey( } } } 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 { @@ -533,9 +588,9 @@ private void remove( TimelineEntry removed = timeline.get(interval); if (removed == null) { - Iterator> iter = timeline.entrySet().iterator(); + Iterator> iter = timeline.entrySet().iterator(); while (iter.hasNext()) { - Map.Entry timelineEntry = iter.next(); + Entry timelineEntry = iter.next(); if (timelineEntry.getValue() == entry) { intervalsToRemove.add(timelineEntry.getKey()); } @@ -557,7 +612,7 @@ private void remove( { timeline.remove(interval); - for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { + for (Entry> versionEntry : allTimelineEntries.entrySet()) { if (versionEntry.getKey().overlap(interval) != null) { if (incompleteOk) { add(timeline, versionEntry.getKey(), versionEntry.getValue().lastEntry().getValue()); @@ -576,12 +631,12 @@ private void remove( private List> lookup(Interval interval, boolean incompleteOk) { - List> retVal = new ArrayList>(); + List> retVal = new ArrayList<>(); NavigableMap timeline = (incompleteOk) ? incompletePartitionsTimeline : completePartitionsTimeline; - for (Map.Entry entry : timeline.entrySet()) { + for (Entry 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 000000000000..9311b9846076 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -0,0 +1,181 @@ +/* + * 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; + +class AtomicUpdateGroup> implements Overshadowable> +{ + // TODO: map?? + private final List> chunks = new ArrayList<>(); + + public AtomicUpdateGroup(PartitionChunk chunk) + { + this.chunks.add(chunk); + } + + public void add(PartitionChunk chunk) + { + if (isFull()) { + throw new IAE("Can't add more chunk[%s] to atomicUpdateGroup[%s]", chunk, chunks); + } + if (!isSameAtomicUpdateGroup(chunks.get(0), chunk)) { + throw new IAE("Can't add chunk[%s] to a different atomicUpdateGroup[%s]", chunk, chunks); + } + if (replaceChunkWith(chunk) == null) { + chunks.add(chunk); + } + } + + public void remove(PartitionChunk 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 chunks.size() == chunks.get(0).getObject().getAtomicUpdateGroupSize(); + } + + public boolean isEmpty() + { + return chunks.isEmpty(); + } + + public List> getChunks() + { + return chunks; + } + + @Nullable + public PartitionChunk findChunk(int partitionId) + { + return chunks.stream().filter(chunk -> chunk.getChunkNumber() == partitionId).findFirst().orElse(null); + } + + @Nullable + public PartitionChunk replaceChunkWith(PartitionChunk newChunk) + { + PartitionChunk oldChunk = null; + for (int i = 0; i < chunks.size(); i++) { + if (newChunk.getChunkNumber() == chunks.get(i).getChunkNumber()) { + oldChunk = chunks.set(i, newChunk); + break; + } + } + return oldChunk; + } + + @Override + public int getStartRootPartitionId() + { + Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); + return chunks.get(0).getObject().getStartRootPartitionId(); + } + + @Override + public short getStartRootPartitionIdAsShort() + { + Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); + return chunks.get(0).getObject().getStartRootPartitionIdAsShort(); + } + + @Override + public int getEndRootPartitionId() + { + Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); + return chunks.get(0).getObject().getEndRootPartitionId(); + } + + @Override + public short getEndRootPartitionIdAsShort() + { + Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); + return chunks.get(0).getObject().getEndRootPartitionIdAsShort(); + } + + @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 > boolean isSameAtomicUpdateGroup( + PartitionChunk c1, + PartitionChunk 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 0d85423bb322..760558958cd2 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 @@ -45,10 +45,14 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec @JsonIgnore private final List partitionDimensions; + // TODO: check what's the valid partitions. is it (max existing partitionId) + # of new partitions? + // or just # of new partitions? + // or do we need a start partitionId? + @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 partitionDimensions, @JacksonInject ObjectMapper jsonMapper ) @@ -64,6 +68,12 @@ public List getPartitionDimensions() return partitionDimensions; } + @Override + public boolean isCompatible(Class other) + { + return other == HashBasedNumberedShardSpec.class; + } + @Override public boolean isInChunk(long timestamp, InputRow inputRow) { 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 000000000000..eaaa2c7085ec --- /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 partitionDimensions; + private final int numPartitions; + + @JsonCreator + public HashBasedNumberedShardSpecFactory( + @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, + @JsonProperty("numPartitions") int numPartitions + ) + { + this.partitionDimensions = partitionDimensions; + this.numPartitions = numPartitions; + } + + @Nullable + @JsonProperty + public List 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 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 ec8f7d891a7e..179e8897c6e6 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,11 +19,13 @@ package org.apache.druid.timeline.partition; +import org.apache.druid.timeline.Overshadowable; + /** */ -public class ImmutablePartitionHolder extends PartitionHolder +public class ImmutablePartitionHolder> extends PartitionHolder { - public ImmutablePartitionHolder(PartitionHolder partitionHolder) + public ImmutablePartitionHolder(PartitionHolder partitionHolder) { super(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 986f8fe2a0d2..8855f0151e8c 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 implements PartitionChunk +public class IntegerPartitionChunk implements PartitionChunk { private final Integer start; private final Integer end; private final int chunkNumber; private final T object; - public static IntegerPartitionChunk make(Integer start, Integer end, int chunkNumber, T obj) + public static IntegerPartitionChunk make(Integer start, Integer end, int chunkNumber, T obj) { return new IntegerPartitionChunk(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 d0e60c2c5175..1ebb24e16038 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 @@ -67,6 +67,12 @@ public boolean possibleInDomain(Map> domain) return true; } + @Override + public boolean isCompatible(Class other) + { + return other == LinearShardSpec.class; + } + @Override public PartitionChunk createChunk(T obj) { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpecFactory.java b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpecFactory.java new file mode 100644 index 000000000000..b6340ec252ff --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpecFactory.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.timeline.partition; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import javax.annotation.Nullable; + +public class LinearShardSpecFactory implements ShardSpecFactory +{ + private static final LinearShardSpecFactory INSTANCE = new LinearShardSpecFactory(); + + public static LinearShardSpecFactory instance() + { + return INSTANCE; + } + + private LinearShardSpecFactory() + { + } + + @Override + public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) + { + return new LinearShardSpec( + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1 + ); + } + + @Override + public ShardSpec create(ObjectMapper objectMapper, int partitionId) + { + return new LinearShardSpec(partitionId); + } + + @Override + public Class getShardSpecClass() + { + 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 7fb2aab3a7a0..dde92167362a 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 @@ private NoneShardSpec() @Override public PartitionChunk createChunk(T obj) { - return new SingleElementPartitionChunk(obj); + return new SingleElementPartitionChunk<>(obj); } @Override @@ -83,6 +83,12 @@ public boolean possibleInDomain(Map> domain) return true; } + @Override + public boolean isCompatible(Class other) + { + return other == NoneShardSpec.class; + } + @Override public boolean equals(Object obj) { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.java b/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.java new file mode 100644 index 000000000000..27cca23cd3f6 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.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.databind.ObjectMapper; + +import javax.annotation.Nullable; + +public class NoneShardSpecFactory implements ShardSpecFactory +{ + private static NoneShardSpecFactory INSTANCE = new NoneShardSpecFactory(); + + public static NoneShardSpecFactory instance() + { + return INSTANCE; + } + + private NoneShardSpecFactory() + { + } + + @Override + public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) + { + return NoneShardSpec.instance(); + } + + @Override + public ShardSpec create(ObjectMapper objectMapper, int partitionId) + { + return NoneShardSpec.instance(); + } + + @Override + public Class getShardSpecClass() + { + return NoneShardSpec.class; + } +} 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 000000000000..dbe1b1060ac5 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java @@ -0,0 +1,115 @@ +/* + * 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; + +public class NumberedOverwritingPartitionChunk implements OverwritingPartitionChunk +{ + private final int chunkId; + private final T object; + + public NumberedOverwritingPartitionChunk(int chunkId, T object) + { + Preconditions.checkArgument( + chunkId >= ShardSpec.NON_ROOT_GEN_START_PARTITION_ID && chunkId < ShardSpec.NON_ROOT_GEN_END_PARTITION_ID, + "partitionNum[%s] >= %s && partitionNum[%s] < %s", + chunkId, + ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, + chunkId, + ShardSpec.NON_ROOT_GEN_END_PARTITION_ID + ); + + this.chunkId = chunkId; + this.object = object; + } + + @Override + public T getObject() + { + return object; + } + + @Override + public boolean abuts(PartitionChunk 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 o) + { + if (o instanceof NumberedOverwritingPartitionChunk) { + final NumberedOverwritingPartitionChunk that = (NumberedOverwritingPartitionChunk) o; + return Integer.compare(chunkId, that.chunkId); + } else { + throw new IAE("Cannot compare against [%s]", o.getClass().getCanonicalName()); + } + } + + @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/NumberedOverwritingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java new file mode 100644 index 000000000000..9f7857fa57ed --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java @@ -0,0 +1,224 @@ +/* + * 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; + +// TODO: abstract numbered?? +public class NumberedOverwritingShardSpec implements OverwritingShardSpec +{ + 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 NumberedOverwritingShardSpec( + @JsonProperty("partitionId") int partitionId, + @JsonProperty("startRootPartitionId") int startRootPartitionId, + @JsonProperty("endRootPartitionId") int endRootPartitionId, + @JsonProperty("minorVersion") short minorVersion, + @JsonProperty("atomicUpdateGroupSize") short atomicUpdateGroupSize + ) + { + Preconditions.checkArgument( + partitionId >= NON_ROOT_GEN_START_PARTITION_ID && partitionId < NON_ROOT_GEN_END_PARTITION_ID, + "partitionNum[%s] >= %s && partitionNum[%s] < %s", + partitionId, + NON_ROOT_GEN_START_PARTITION_ID, + partitionId, + NON_ROOT_GEN_END_PARTITION_ID + ); + Preconditions.checkArgument( + startRootPartitionId >= ROOT_GEN_START_PARTITION_ID && startRootPartitionId < ROOT_GEN_END_PARTITION_ID, + "startRootPartitionId[%s] >= %s && startRootPartitionId[%s] < %s", + startRootPartitionId, + ROOT_GEN_START_PARTITION_ID, + startRootPartitionId, + ROOT_GEN_END_PARTITION_ID + ); + Preconditions.checkArgument( + endRootPartitionId >= ROOT_GEN_START_PARTITION_ID && endRootPartitionId < ROOT_GEN_END_PARTITION_ID, + "endRootPartitionId[%s] >= %s && endRootPartitionId[%s] < %s", + endRootPartitionId, + ROOT_GEN_START_PARTITION_ID, + endRootPartitionId, + ROOT_GEN_END_PARTITION_ID + ); + Preconditions.checkArgument(minorVersion > 0, "minorVersion[%s] > 0", minorVersion); + Preconditions.checkArgument( + atomicUpdateGroupSize > 0 || atomicUpdateGroupSize == ShardSpec.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE, + "atomicUpdateGroupSize[%s] > 0 or == %s", + atomicUpdateGroupSize, + ShardSpec.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE + ); + + this.partitionId = partitionId; + this.startRootPartitionId = (short) startRootPartitionId; + this.endRootPartitionId = (short) endRootPartitionId; + this.minorVersion = minorVersion; + this.atomicUpdateGroupSize = atomicUpdateGroupSize; + } + + public NumberedOverwritingShardSpec( + int partitionId, + int startRootPartitionId, + int endRootPartitionId, + short minorVersion + ) + { + this( + partitionId, + startRootPartitionId, + endRootPartitionId, + minorVersion, + ShardSpec.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE + ); + } + + @Override + public OverwritingShardSpec withAtomicUpdateGroupSize(short atomicUpdateGroupSize) + { + return new NumberedOverwritingShardSpec( + this.partitionId, + this.startRootPartitionId, + this.endRootPartitionId, + this.minorVersion, + atomicUpdateGroupSize + ); + } + + @Override + public PartitionChunk 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 shardSpecs) + { + return (long timestamp, InputRow row) -> shardSpecs.get(0); + } + + @Override + public List getDomainDimensions() + { + return Collections.emptyList(); + } + + @Override + public boolean possibleInDomain(Map> domain) + { + return true; + } + + @Override + public boolean isCompatible(Class other) + { + return other == NumberedOverwritingShardSpec.class || other == NumberedShardSpec.class; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + NumberedOverwritingShardSpec that = (NumberedOverwritingShardSpec) 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 "NumberedOverwritingShardSpec{" + + "partitionNum=" + partitionId + + ", startRootPartitionId=" + startRootPartitionId + + ", endRootPartitionId=" + endRootPartitionId + + ", minorVersion=" + minorVersion + + ", atomicUpdateGroupSize=" + atomicUpdateGroupSize + + '}'; + } +} 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 000000000000..81ffb48b6853 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java @@ -0,0 +1,88 @@ +/* + * 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) + { + return new NumberedOverwritingShardSpec( + specOfPreviousMaxPartitionId == null + ? ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + : specOfPreviousMaxPartitionId.getPartitionNum() + 1, + startRootPartitionId, + endRootPartitionId, + minorVersion + ); + } + + @Override + public ShardSpec create(ObjectMapper objectMapper, int partitionId) + { + return new NumberedOverwritingShardSpec(partitionId, startRootPartitionId, endRootPartitionId, minorVersion); + } + + @Override + public Class getShardSpecClass() + { + return NumberedOverwritingShardSpec.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 5ab89f6e8084..2d32532bece0 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 int hashCode() { 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 8b555e26fe80..9bc68809b40d 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 boolean possibleInDomain(Map> domain) return true; } + @Override + public boolean isCompatible(Class other) + { + return other == NumberedShardSpec.class || other == NumberedOverwritingShardSpec.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 000000000000..486f9edcaa00 --- /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 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 000000000000..f7e5975781aa --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -0,0 +1,552 @@ +/* + * 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.shorts.Short2ObjectMap; +import it.unimi.dsi.fastutil.shorts.Short2ObjectRBTreeMap; +import it.unimi.dsi.fastutil.shorts.Short2ObjectSortedMap; +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.Collection; +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.TreeMap; +import java.util.stream.Collectors; + +// TODO: rename + +/** + * Not thread-safe + * @param + */ +public class OvershadowableManager> +{ + 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> knownPartitionChunks; // served segments + + // start partitionId -> end partitionId -> minorVersion -> atomicUpdateGroup + private final TreeMap>> standbyGroups; + private final TreeMap>> visibleGroup; // TODO: singleton navigable map + private final TreeMap>> overshadowedGroups; + + public OvershadowableManager() + { + this.knownPartitionChunks = new HashMap<>(); + this.standbyGroups = new TreeMap<>(); + this.visibleGroup = new TreeMap<>(); + this.overshadowedGroups = new TreeMap<>(); + } + + public OvershadowableManager(OvershadowableManager 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>> 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 void transitPartitionChunkState(AtomicUpdateGroup atomicUpdateGroup, State from, State to) + { + Preconditions.checkNotNull(atomicUpdateGroup, "atomicUpdateGroup"); + Preconditions.checkArgument(!atomicUpdateGroup.isEmpty(), "empty atomicUpdateGroup"); + + removeFrom(atomicUpdateGroup, from); + addTo(atomicUpdateGroup, to); + } + + @Nullable + private AtomicUpdateGroup searchForStateOf(PartitionChunk chunk, State state) + { + final Short2ObjectSortedMap> versionToGroup = getStateMap(state).get( + RootPartitionRange.of(chunk) + ); + if (versionToGroup != null) { + final AtomicUpdateGroup 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 tryRemoveFromState(PartitionChunk chunk, State state) + { + final RootPartitionRange rangeKey = RootPartitionRange.of(chunk); + final Short2ObjectSortedMap> versionToGroup = getStateMap(state).get(rangeKey); + if (versionToGroup != null) { + final AtomicUpdateGroup 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); + } + } + + handleRemove(atomicUpdateGroup, RootPartitionRange.of(chunk), chunk.getObject().getMinorVersion(), state); + return atomicUpdateGroup; + } + } + return null; + } + + private List>> findOvershadowedBy( + AtomicUpdateGroup aug, + State fromState + ) + { + final RootPartitionRange rangeKeyOfGivenAug = RootPartitionRange.of(aug); + return findOvershadowedBy(rangeKeyOfGivenAug, aug.getMinorVersion(), fromState); + } + + private List>> findOvershadowedBy( + RootPartitionRange rangeOfAug, + short minorVersion, + State fromState + ) + { + Entry>> current = getStateMap(fromState) + .floorEntry(rangeOfAug); + + if (current == null) { + return Collections.emptyList(); + } + + // Find the first key for searching for overshadowed atomicUpdateGroup + while (true) { + final Entry>> lowerEntry = getStateMap(fromState) + .lowerEntry(current.getKey()); + if (lowerEntry != null && lowerEntry.getKey().startPartitionId == rangeOfAug.startPartitionId) { + current = lowerEntry; + } else { + break; + } + } + + final List>> found = new ArrayList<>(); + while (current != null && rangeOfAug.contains(current.getKey())) { + final Short2ObjectSortedMap> versionToGroup = current.getValue(); + found.addAll(versionToGroup.subMap(versionToGroup.firstShortKey(), minorVersion).short2ObjectEntrySet()); + current = getStateMap(fromState).higherEntry(current.getKey()); + } + return found; + } + + /** + * Handles addition of the atomicUpdateGroup to the given state + */ + private void handleAdd(AtomicUpdateGroup aug, State newStateOfAug) + { + if (newStateOfAug == 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 -> transitPartitionChunkState(entry.getValue(), State.VISIBLE, State.OVERSHADOWED)); + transitPartitionChunkState(aug, State.STANDBY, State.VISIBLE); + } + } + } + + private void addTo(AtomicUpdateGroup aug, State state) + { + final AtomicUpdateGroup existing = getStateMap(state) + .computeIfAbsent(RootPartitionRange.of(aug), k -> new Short2ObjectRBTreeMap<>()) + .put(aug.getMinorVersion(), aug); + + if (existing != null) { + throw new ISE("AtomicUpdateGroup[%s] is already in state[%s]", aug, state); + } + + handleAdd(aug, state); + } + + public void add(PartitionChunk chunk) + { + final PartitionChunk existingChunk = knownPartitionChunks.put(chunk.getChunkNumber(), chunk); + if (existingChunk != null && !existingChunk.equals(chunk)) { + throw new ISE( + "existingChunk[%s] is different from newChunk[%s] for partitionId[%d]", + existingChunk, + chunk, + chunk.getChunkNumber() + ); + } + + // Find atomicUpdateGroup of the new chunk + AtomicUpdateGroup atomicUpdateGroup = searchForStateOf(chunk, State.OVERSHADOWED); + + if (atomicUpdateGroup != null) { + atomicUpdateGroup.add(chunk); + } else { + atomicUpdateGroup = searchForStateOf(chunk, State.STANDBY); + + if (atomicUpdateGroup != null) { + atomicUpdateGroup.add(chunk); + handleAdd(atomicUpdateGroup, State.STANDBY); + } else { + atomicUpdateGroup = searchForStateOf(chunk, State.VISIBLE); + + if (atomicUpdateGroup != null) { + // A new chunk of the same major version and partitionId can be added in segment handoff + // from stream ingestion tasks to historicals + final PartitionChunk existing = atomicUpdateGroup.replaceChunkWith(chunk); + if (existing == null) { + throw new ISE( + "Can't add a new partitionChunk[%s] to a visible atomicUpdateGroup[%s]", + chunk, + atomicUpdateGroup + ); + } else if (!chunk.equals(existing)) { + throw new ISE( + "WTH? a new partitionChunk[%s] has the same partitionId but different from existing chunk[%s]", + chunk, + existing + ); + } + } else { + final AtomicUpdateGroup 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.isOvershadow(newAtomicUpdateGroup)); + + if (overshadowed) { + addTo(newAtomicUpdateGroup, State.OVERSHADOWED); + } else { + addTo(newAtomicUpdateGroup, State.STANDBY); + } + } + } + } + } + + /** + * Handles of removal of an empty atomicUpdateGroup from a state. + */ + private void handleRemove( + AtomicUpdateGroup augOfRemovedChunk, + RootPartitionRange rangeOfAug, + short minorVersion, + State stateOfRemovedAug + ) + { + if (stateOfRemovedAug == State.STANDBY) { + // If an atomicUpdateGroup is overshadowed by another standby atomicUpdateGroup, there must be another visible + // atomicUpdateGroup which also overshadows the same atomicUpdateGroup. + // As a result, the state of overshadowed atomicUpdateGroup shouldn't be changed and we do nothing here. + + } else 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> latestFullAugs = findLatestFullyAvailableOvershadowedAtomicUpdateGroup( + rangeOfAug, + minorVersion + ); + + if (!latestFullAugs.isEmpty()) { + // Move the atomicUpdateGroup to standby + // and move the fully available overshadowed atomicUpdateGroup to visible + if (!augOfRemovedChunk.isEmpty()) { + transitPartitionChunkState(augOfRemovedChunk, State.VISIBLE, State.STANDBY); + } + latestFullAugs.forEach(group -> transitPartitionChunkState(group, State.OVERSHADOWED, State.VISIBLE)); + } + } else { + // do nothing + } + } + + private List> findLatestFullyAvailableOvershadowedAtomicUpdateGroup( + RootPartitionRange rangeOfAug, + short minorVersion + ) + { + final List>> overshadowedGroups = findOvershadowedBy( + rangeOfAug, + minorVersion, + State.OVERSHADOWED + ); + if (overshadowedGroups.isEmpty()) { + return Collections.emptyList(); + } + + final OvershadowableManager manager = new OvershadowableManager<>(); + overshadowedGroups.stream() + .flatMap(entry -> entry.getValue().getChunks().stream()) + .forEach(manager::add); + + return manager.visibleGroup + .values() + .stream() + .flatMap(versionToGroup -> versionToGroup.values().stream()) + .collect(Collectors.toList()); + } + + private void removeFrom(AtomicUpdateGroup aug, State state) + { + final RootPartitionRange rangeKey = RootPartitionRange.of(aug); + final Short2ObjectSortedMap> versionToGroup = getStateMap(state).get(rangeKey); + if (versionToGroup == null) { + throw new ISE("Unknown atomicUpdateGroup[%s] in state[%s]", aug, state); + } + + final AtomicUpdateGroup 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 remove(PartitionChunk partitionChunk) + { + final PartitionChunk 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 augOfRemovedChunk = tryRemoveFromState(partitionChunk, State.STANDBY); + + if (augOfRemovedChunk == null) { + augOfRemovedChunk = tryRemoveFromState(partitionChunk, State.VISIBLE); + if (augOfRemovedChunk == null) { + augOfRemovedChunk = tryRemoveFromState(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 getChunk(int partitionId) + { + final PartitionChunk chunk = knownPartitionChunks.get(partitionId); + if (chunk == null) { + return null; + } + final AtomicUpdateGroup aug = searchForStateOf(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> getVisibles() + { + return visibleGroup.values() + .stream() + .flatMap(treeMap -> treeMap.values().stream()) + .flatMap(aug -> aug.getChunks().stream()) + .collect(Collectors.toList()); + } + + public Collection> getOvershadowed() + { + return overshadowedGroups.values() + .stream() + .flatMap(treeMap -> treeMap.values().stream()) + .flatMap(aug -> aug.getChunks().stream()) + .collect(Collectors.toList()); + } + + @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 + { + private final short startPartitionId; + private final short endPartitionId; + + private static > RootPartitionRange of(PartitionChunk chunk) + { + return of(chunk.getObject().getStartRootPartitionId(), chunk.getObject().getEndRootPartitionId()); + } + + private static > RootPartitionRange of(AtomicUpdateGroup 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 + + '}'; + } + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OverwritingPartitionChunk.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwritingPartitionChunk.java new file mode 100644 index 000000000000..86b2651e6ea4 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwritingPartitionChunk.java @@ -0,0 +1,25 @@ +/* + * 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; + +// TODO: rename +public interface OverwritingPartitionChunk extends PartitionChunk +{ +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java new file mode 100644 index 000000000000..74577b688faa --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java @@ -0,0 +1,31 @@ +/* + * 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; + +// TODO: rename +public interface OverwritingShardSpec extends ShardSpec +{ + default OverwritingShardSpec withAtomicUpdateGroupSize(int atomicUpdateGroupSize) + { + return withAtomicUpdateGroupSize((short) atomicUpdateGroupSize); + } + + OverwritingShardSpec 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 396b4f3bdd63..4c70061e1b16 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,74 +20,64 @@ 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 java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.SortedSet; +import java.util.Objects; import java.util.Spliterator; -import java.util.TreeSet; +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 implements Iterable> +public class PartitionHolder> implements Iterable> { - private final TreeSet> holderSet; + private final OvershadowableManager overshadowableManager; public PartitionHolder(PartitionChunk initialChunk) { - this.holderSet = new TreeSet<>(); + this.overshadowableManager = new OvershadowableManager<>(); add(initialChunk); } public PartitionHolder(List> initialChunks) { - this.holderSet = new TreeSet<>(); + this.overshadowableManager = new OvershadowableManager<>(); for (PartitionChunk chunk : initialChunks) { add(chunk); } } - public PartitionHolder(PartitionHolder partitionHolder) + public PartitionHolder(PartitionHolder partitionHolder) { - this.holderSet = new TreeSet<>(); - this.holderSet.addAll(partitionHolder.holderSet); + this.overshadowableManager = new OvershadowableManager<>(partitionHolder.overshadowableManager); } public void add(PartitionChunk chunk) { - holderSet.add(chunk); + overshadowableManager.add(chunk); } public PartitionChunk remove(PartitionChunk chunk) { - if (!holderSet.isEmpty()) { - // Somewhat funky implementation in order to return the removed object as it exists in the set - SortedSet> tailSet = holderSet.tailSet(chunk, true); - if (!tailSet.isEmpty()) { - PartitionChunk element = tailSet.first(); - if (chunk.equals(element)) { - holderSet.remove(element); - return element; - } - } - } - return null; + return overshadowableManager.remove(chunk); } public boolean isEmpty() { - return holderSet.isEmpty(); + return overshadowableManager.isEmpty(); } public boolean isComplete() { - if (holderSet.isEmpty()) { + if (overshadowableManager.isEmpty()) { return false; } - Iterator> iter = holderSet.iterator(); + Iterator> iter = iterator(); PartitionChunk curr = iter.next(); @@ -96,7 +86,7 @@ public boolean isComplete() } if (curr.isEnd()) { - return true; + return overshadowableManager.isComplete(); } while (iter.hasNext()) { @@ -106,7 +96,7 @@ public boolean isComplete() } if (next.isEnd()) { - return true; + return overshadowableManager.isComplete(); } curr = next; } @@ -116,24 +106,29 @@ public boolean isComplete() public PartitionChunk getChunk(final int partitionNum) { - final Iterator> retVal = Iterators.filter( - holderSet.iterator(), - input -> input.getChunkNumber() == partitionNum - ); - - return retVal.hasNext() ? retVal.next() : null; + return overshadowableManager.getChunk(partitionNum); } @Override public Iterator> iterator() { - return holderSet.iterator(); + return overshadowableManager.getVisibles().iterator(); } @Override public Spliterator> spliterator() { - return holderSet.spliterator(); + return overshadowableManager.getVisibles().spliterator(); + } + + public Stream> stream() + { + return StreamSupport.stream(spliterator(), false); + } + + public Collection> getOvershadowed() + { + return overshadowableManager.getOvershadowed(); } public Iterable payloads() @@ -150,27 +145,21 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - - PartitionHolder that = (PartitionHolder) o; - - if (!holderSet.equals(that.holderSet)) { - return false; - } - - return true; + PartitionHolder that = (PartitionHolder) o; + return Objects.equals(overshadowableManager, that.overshadowableManager); } @Override public int hashCode() { - return holderSet.hashCode(); + return Objects.hash(overshadowableManager); } @Override public String toString() { return "PartitionHolder{" + - "holderSet=" + holderSet + + "overshadowableManager=" + overshadowableManager + '}'; } } 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 10a6d6cf50d0..5ddcc9e5154c 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,16 +37,44 @@ @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 = NumberedOverwritingShardSpec.class) }) public interface ShardSpec { + // TODO: move to somewhere..? + int ROOT_GEN_START_PARTITION_ID = 0; + int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive + int NON_ROOT_GEN_START_PARTITION_ID = 32768; + int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive + short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; + PartitionChunk createChunk(T obj); boolean isInChunk(long timestamp, InputRow inputRow); 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 shardSpecs); /** @@ -61,4 +89,6 @@ public interface ShardSpec * @return possibility of in domain */ boolean possibleInDomain(Map> domain); + + boolean isCompatible(Class 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 000000000000..5da2bd5f73b0 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java @@ -0,0 +1,42 @@ +/* + * 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; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes({ + @JsonSubTypes.Type(name = "none", value = NoneShardSpecFactory.class), + @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 +{ + ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId); + + ShardSpec create(ObjectMapper objectMapper, int partitionId); + + Class 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 f2b84413f171..968a1d74cc98 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.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 SingleDimensionShardSpec( @JsonProperty("partitionNum") int partitionNum ) { + Preconditions.checkArgument(partitionNum >= 0, "partitionNum >= 0"); this.dimension = Preconditions.checkNotNull(dimension, "dimension"); this.start = start; this.end = end; @@ -131,6 +136,12 @@ public boolean possibleInDomain(Map> domain) return !rangeSet.subRangeSet(getRange()).isEmpty(); } + @Override + public boolean isCompatible(Class other) + { + return other == SingleDimensionShardSpec.class; + } + @Override public PartitionChunk createChunk(T 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 f12073fccb39..2567fe617552 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 boolean equals(Object o) @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 ff7171f571d3..28d7505e70a6 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 int hashCode() 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 701578541c64..bc3cf902b9ed 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.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 boolean possibleInDomain(Map> domain) { return true; } + + @Override + public boolean isCompatible(Class other) + { + return false; + } }; } @@ -117,7 +124,7 @@ public void testV1Serialization() throws Exception loadSpec, Arrays.asList("dim1", "dim2"), Arrays.asList("met1", "met2"), - NoneShardSpec.instance(), + new NumberedShardSpec(3, 0), TEST_VERSION, 1 ); @@ -134,7 +141,7 @@ public void testV1Serialization() throws Exception 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 681f6a605c5c..986e36aaca1f 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.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.ShardSpec; import org.apache.druid.timeline.partition.SingleElementPartitionChunk; import org.joda.time.DateTime; import org.joda.time.Days; @@ -45,13 +48,14 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.Set; /** */ public class VersionedIntervalTimelineTest { - VersionedIntervalTimeline timeline; + VersionedIntervalTimeline timeline; @Before public void setUp() @@ -70,8 +74,8 @@ public void setUp() 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(0, 20))); + add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger(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); @@ -168,7 +172,7 @@ public void testMay() @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(9))); assertValues( Arrays.asList( createExpected("2011-05-01/2011-05-03", "2", 7), @@ -227,35 +231,35 @@ public void testInsertInWrongOrder() @Test public void testRemove() { - for (TimelineObjectHolder holder : timeline.findOvershadowed()) { - for (PartitionChunk chunk : holder.getObject()) { + for (TimelineObjectHolder holder : timeline.findFullyOvershadowed()) { + for (PartitionChunk 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))), 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))), 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))), 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))), timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1") ); @@ -279,7 +283,7 @@ public void testFindEntryWithOverlap() add("2011-01-02/2011-01-05", "2", 1); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-01-02T02/2011-01-04"), "1") ); } @@ -293,8 +297,8 @@ public void testPartitioning() 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(0, 20)), + IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger(1, 21)) ) ), createExpected("2011-10-03/2011-10-04", "3", 3), @@ -310,36 +314,36 @@ public void testPartialPartitionNotReturned() { 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(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(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(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(0, 60)), + IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger(1, 61)), + IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger(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 void testIncompletePartitionDoesNotOvershadow() { 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(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(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(2, 62))); assertValues( ImmutableSet.of( createExpected("2011-10-05/2011-10-06", "5", 5) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -367,13 +371,13 @@ public void testRemovePartitionMakesIncomplete() { testIncompletePartitionDoesNotOvershadow(); - final IntegerPartitionChunk chunk = IntegerPartitionChunk.make(null, 10, 0, 60); + final IntegerPartitionChunk chunk = IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger(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 @@ -429,6 +433,8 @@ public void testOverlapSameVersionThrowException() add("2011-01-05/2011-01-15", "1", 3); } + // 2|----| + // 2|----| // 1|----| // 1|----| @Test @@ -443,7 +449,7 @@ public void testOverlapSameVersionIsOkay() assertValues( Collections.singletonList( - createExpected("2011-01-01/2011-01-10", "2", 2) + createExpected("2011-01-01/2011-01-10", "2", 3) ), timeline.lookup(Intervals.of("2011-01-01/2011-01-10")) ); @@ -560,8 +566,8 @@ public void testOverlapSecondAfter() 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 +936,7 @@ public void testOverlapV1Large() 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 +955,8 @@ public void testOverlapV2Large() 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 +1115,7 @@ public void testOverlapOvershadowedThirdContains() createExpected("2011-01-03/2011-01-06", "1", 1), createExpected("2011-01-09/2011-01-12", "1", 2) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1132,7 +1138,7 @@ public void testOverlapOvershadowedAligned() createExpected("2011-01-05/2011-01-10", "2", 2), createExpected("2011-01-01/2011-01-10", "1", 3) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1154,7 +1160,7 @@ public void testOverlapOvershadowedSomeComplexOverlapsCantThinkOfBetterName() createExpected("2011-01-03/2011-01-12", "1", 3), createExpected("2011-01-01/2011-01-05", "2", 1) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1308,7 +1314,7 @@ public void testOvershadowingHigherVersionWins1() createExpected("2011-04-03/2011-04-06", "1", 3), createExpected("2011-04-06/2011-04-09", "1", 4) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1326,7 +1332,7 @@ public void testOvershadowingHigherVersionWins2() ImmutableSet.of( createExpected("2011-04-01/2011-04-09", "1", 1) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1346,7 +1352,7 @@ public void testOvershadowingHigherVersionWins3() createExpected("2011-04-03/2011-04-06", "1", 3), createExpected("2011-04-09/2011-04-12", "1", 4) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1364,7 +1370,7 @@ public void testOvershadowingHigherVersionWins4() createExpected("2011-04-03/2011-04-06", "1", 3), createExpected("2011-04-06/2011-04-09", "1", 4) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1379,7 +1385,7 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower1() assertValues( ImmutableSet.of(), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1394,7 +1400,7 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower2() assertValues( ImmutableSet.of(), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1409,7 +1415,7 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower3() assertValues( ImmutableSet.of(), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1427,7 +1433,7 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower4() ImmutableSet.of( createExpected("2011-04-03/2011-04-06", "1", 3) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1447,7 +1453,7 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower5() createExpected("2011-04-03/2011-04-06", "1", 3), createExpected("2011-04-09/2011-04-12", "1", 3) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1465,7 +1471,7 @@ public void testOvershadowingSameIntervalHighVersionWins() createExpected("2011-04-01/2011-04-09", "2", 3), createExpected("2011-04-01/2011-04-09", "1", 1) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1484,7 +1490,7 @@ public void testOvershadowingSameIntervalSameVersionAllKept() createExpected("2011-04-01/2011-04-09", "2", 3), createExpected("2011-04-01/2011-04-09", "1", 1) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); } @@ -1504,30 +1510,30 @@ public void testRemoveIncompleteKeepsComplete() { 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(0, 77))); + add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(1, 88))); + add("2011-04-01/2011-04-02", "2", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger(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(0, 77)), + IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(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(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(0, 77)), + IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(1, 88)) ) ) ), @@ -1537,11 +1543,11 @@ public void testRemoveIncompleteKeepsComplete() 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(0, 99)) ) ) ), - timeline.findOvershadowed() + timeline.findFullyOvershadowed() ); testRemove(); @@ -1550,8 +1556,8 @@ public void testRemoveIncompleteKeepsComplete() 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(0, 77)), + IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(1, 88)) ) ) ), @@ -1564,64 +1570,64 @@ public void testIsOvershadowedWithNonOverlappingSegmentsInTimeline() { 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)); + add("2011-04-07/2011-04-09", "1", makeSingle(1)); + + add("2011-04-15/2011-04-17", "1", makeSingle(1)); + add("2011-04-17/2011-04-19", "1", makeSingle(1)); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "1", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "1", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "1", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "1", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "2", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "2", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "2", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "2", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-30"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-08"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-30"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-30"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-15"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-17"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-19"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-30"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-16"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-17"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-18"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-19"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-20"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-30"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-19/2011-04-20"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0", new OvershadowableInteger(0, 1))); } @Test @@ -1629,87 +1635,304 @@ public void testIsOvershadowedWithOverlappingSegmentsInTimeline() { 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>> createExpected( + add("2011-04-05/2011-04-09", "11", makeSingle(1)); + add("2011-04-07/2011-04-11", "12", makeSingle(1)); + + add("2011-04-15/2011-04-19", "12", makeSingle(1)); + add("2011-04-17/2011-04-21", "11", makeSingle(1)); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-11"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "12", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "12", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "12", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "12", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "12", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "12", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "13", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "13", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "13", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "13", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "13", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "13", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-12"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-15"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-16"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-17"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-18"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-19"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-20"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-21"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-22"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0", new OvershadowableInteger(0, 1))); + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-11"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-12"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-15"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-16"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-17"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-18"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-19"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-20"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-21"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-22"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-15"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-16"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-17"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-18"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-19"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-20"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-21"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-22"), "0", new OvershadowableInteger(0, 1))); + + Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-21"), "0", new OvershadowableInteger(0, 1))); + Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0", new OvershadowableInteger(0, 1))); + } + + @Test + public void testOvershadowedByReference() + { + timeline = makeStringIntegerTimeline(); + + add("2019-01-01/2019-01-02", "0", makeNumbered(0, 0)); + add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); + add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(0, 1, 0, 3, 1, 2)); + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(1, 1, 0, 3, 1, 2)); + + Assert.assertEquals( + ImmutableSet.of( + makeTimelineObjectHolder( + "2019-01-01/2019-01-02", + "0", + ImmutableList.of(makeNumbered(0, 0), makeNumbered(1, 0), makeNumbered(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)); + add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); + add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + + // 2019-01-02/2019-01-03 + add("2019-01-02/2019-01-03", "0", makeNumbered(0, 0)); + add("2019-01-02/2019-01-03", "0", makeNumbered(1, 0)); + + // Overwrite 2019-01-01/2019-01-02 + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(0, 1, 0, 3, 1, 2)); + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(1, 1, 0, 3, 1, 2)); + + // Overwrite 2019-01-01/2019-01-02 + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(2, 2, 0, 3, 2, 2)); + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(3, 2, 0, 3, 2, 2)); + + Assert.assertEquals( + ImmutableSet.of( + makeTimelineObjectHolder( + "2019-01-01/2019-01-02", + "0", + ImmutableList.of( + makeNumbered(0, 0), + makeNumbered(1, 0), + makeNumbered(2, 0), + makeNumberedOverwriting(0, 1, 0, 3, 1, 2), + makeNumberedOverwriting(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)); + add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); + add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + + // 2019-01-02/2019-01-03 + add("2019-01-02/2019-01-03", "0", makeNumbered(0, 0)); + add("2019-01-02/2019-01-03", "0", makeNumbered(1, 0)); + + // Overwrite 2019-01-01/2019-01-02 + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(0, 1, 0, 3, 1, 2)); + add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(1, 1, 0, 3, 1, 2)); + + // Overwrite 2019-01-01/2019-01-02 + add("2019-01-01/2019-01-02", "1", makeNumbered(0, 0)); + add("2019-01-01/2019-01-02", "1", makeNumbered(1, 0)); + + Assert.assertEquals( + ImmutableSet.of( + makeTimelineObjectHolder( + "2019-01-01/2019-01-02", + "0", + ImmutableList.of( + makeNumbered(0, 0), + makeNumbered(1, 0), + makeNumbered(2, 0), + makeNumberedOverwriting(0, 1, 0, 3, 1, 2), + makeNumberedOverwriting(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)); + add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); + add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + + // 2019-01-02/2019-01-03 + add("2019-01-02/2019-01-03", "0", makeNumbered(0, 0)); + add("2019-01-02/2019-01-03", "0", makeNumbered(1, 0)); + + // Overwrite 2019-01-01/2019-01-02 + add("2019-01-01/2019-01-02", "1", makeNumbered(0, 0)); + add("2019-01-01/2019-01-02", "1", makeNumbered(1, 0)); + + // Overwrite 2019-01-01/2019-01-02 + add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(0, 1, 0, 2, 1, 3)); + add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(1, 1, 0, 2, 1, 3)); + add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(2, 1, 0, 2, 1, 3)); + + Assert.assertEquals( + ImmutableSet.of( + makeTimelineObjectHolder( + "2019-01-01/2019-01-02", + "0", + ImmutableList.of( + makeNumbered(0, 0), + makeNumbered(1, 0), + makeNumbered(2, 0) + ) + ), + makeTimelineObjectHolder( + "2019-01-01/2019-01-02", + "1", + ImmutableList.of( + makeNumbered(0, 0), + makeNumbered(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)); + add(interval, "0", makeNumbered(1, 0)); + add(interval, "0", makeNumbered(2, 0)); + + // Overwrite 2019-01-01/2019-01-02 + add(interval, "1", makeNumbered(0, 0)); + add(interval, "1", makeNumbered(1, 0)); + + // Overwrite 2019-01-01/2019-01-02 + add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(0, 1, 0, 2, 1, 3)); + add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(1, 1, 0, 2, 1, 3)); + add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(2, 1, 0, 2, 1, 3)); + + timeline.remove( + interval, + "1", + makeNumberedOverwriting(2, 1, 0, 2, 1, 3) + ); + + final List> holders = timeline.lookup(interval); + + Assert.assertEquals( + ImmutableList.of( + new TimelineObjectHolder<>( + interval, + "1", + new PartitionHolder<>( + ImmutableList.of( + makeNumbered(0, 0), + makeNumbered(1, 0), + makeNumberedOverwriting(0, 1, 0, 2, 1, 3), + makeNumberedOverwriting(1, 1, 0, 2, 1, 3) + ) + ) + ) + ), + holders + ); + } + + // TODO: test if the middle segment is missing (B among A <- B <- C) + + // TODO: test fall back when the middle segment is missing + + // TODO: test isOvershadowed() + + private TimelineObjectHolder makeTimelineObjectHolder( + String interval, + String version, + List> chunks + ) + { + return new TimelineObjectHolder<>( + Intervals.of(interval), + Intervals.of(interval), + version, + new PartitionHolder<>(chunks) + ); + } + + private Pair>> createExpected( String intervalString, String version, Integer value @@ -1722,21 +1945,58 @@ private Pair>> createExpected( ); } - private Pair>> createExpected( + private Pair>> createExpected( String intervalString, String version, - List> values + List> values ) { return Pair.of( Intervals.of(intervalString), - Pair.of(version, new PartitionHolder(values)) + Pair.of(version, new PartitionHolder<>(values)) ); } - private SingleElementPartitionChunk makeSingle(Integer value) + private PartitionChunk makeSingle(int value) { - return new SingleElementPartitionChunk(value); + return makeSingle(0, value); + } + + private PartitionChunk makeSingle(int partitionNum, int val) + { + return new SingleElementPartitionChunk<>(new OvershadowableInteger(partitionNum, val)); + } + + private PartitionChunk makeNumbered(int partitionNum, int val) + { + return new NumberedPartitionChunk<>( + partitionNum, + 0, + new OvershadowableInteger(partitionNum, val) + ); + } + + private PartitionChunk makeNumberedOverwriting( + int partitionNumOrdinal, + int val, + int startRootPartitionId, + int endRootPartitionId, + int minorVersion, + int atomicUpdateGroupSize + ) + { + final int partitionNum = ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + partitionNumOrdinal; + return new NumberedOverwritingPartitionChunk<>( + partitionNum, + new OvershadowableInteger( + partitionNum, + val, + startRootPartitionId, + endRootPartitionId, + minorVersion, + atomicUpdateGroupSize + ) + ); } private void add(String interval, String version, Integer value) @@ -1749,66 +2009,167 @@ private void add(Interval interval, String version, Integer value) add(interval, version, makeSingle(value)); } - private void add(String interval, String version, PartitionChunk value) + private void add(String interval, String version, PartitionChunk value) { add(Intervals.of(interval), version, value); } - private void add(Interval interval, String version, PartitionChunk value) + private void add(Interval interval, String version, PartitionChunk value) { timeline.add(interval, version, value); } private void assertValues( - List>>> expected, - List> actual + List>>> expected, + List> actual ) { Assert.assertEquals("Sizes did not match.", expected.size(), actual.size()); - Iterator>>> expectedIter = expected.iterator(); - Iterator> actualIter = actual.iterator(); + Iterator>>> expectedIter = expected.iterator(); + Iterator> actualIter = actual.iterator(); while (expectedIter.hasNext()) { - Pair>> pair = expectedIter.next(); - TimelineObjectHolder holder = actualIter.next(); + Pair>> pair = expectedIter.next(); + TimelineObjectHolder 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> expectedChunks = Lists.newArrayList(pair.rhs.rhs); + final List> 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>>> expected, - Set> actual + Set>>> expected, + Set> actual ) { Assert.assertEquals("Sizes did not match.", expected.size(), actual.size()); - Set>>> actualSet = + Set>>> actualSet = Sets.newHashSet( Iterables.transform( actual, - new Function, Pair>>>() - { - @Override - public Pair>> apply( - TimelineObjectHolder 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 makeStringIntegerTimeline() + private VersionedIntervalTimeline makeStringIntegerTimeline() { - return new VersionedIntervalTimeline(Ordering.natural()); + return new VersionedIntervalTimeline<>(Ordering.natural()); } + private static class OvershadowableInteger implements Overshadowable + { + 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(int partitionNum, int val) + { + this(partitionNum, val, partitionNum, partitionNum + 1, 0, 1); + } + + private OvershadowableInteger( + int partitionNum, + int val, + int startRootPartitionId, + int endRootPartitionId, + int minorVersion, + int atomicUpdateGroupSize + ) + { + 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; + } + + @Override + public int hashCode() + { + return Objects.hash( + partitionNum, + val, + startRootPartitionId, + endRootPartitionId, + minorVersion, + atomicUpdateGroupSize + ); + } + + @Override + public String toString() + { + return "OvershadowableInteger{" + + "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 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 71806bb9c804..a097ecacf22e 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,19 +19,28 @@ package org.apache.druid.timeline.partition; +import org.apache.druid.timeline.Overshadowable; import org.junit.Assert; import org.junit.Test; -import static org.apache.druid.timeline.partition.IntegerPartitionChunk.make; - /** */ public class IntegerPartitionChunkTest { + private static IntegerPartitionChunk make( + Integer start, + Integer end, + int chunkNumber, + int obj + ) + { + return new IntegerPartitionChunk<>(start, end, chunkNumber, new OvershadowableInteger(obj)); + } + @Test public void testAbuts() { - IntegerPartitionChunk lhs = make(null, 10, 0, 1); + IntegerPartitionChunk lhs = make(null, 10, 0, 1); Assert.assertTrue(lhs.abuts(make(10, null, 1, 2))); Assert.assertFalse(lhs.abuts(make(11, null, 2, 3))); @@ -79,4 +88,38 @@ public void testEquals() 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 + { + private final int val; + + OvershadowableInteger(int val) + { + this.val = val; + } + + @Override + public int getStartRootPartitionId() + { + return 0; + } + + @Override + public int getEndRootPartitionId() + { + return 1; + } + + @Override + public short getMinorVersion() + { + return 0; + } + + @Override + public short getAtomicUpdateGroupSize() + { + return 1; + } + } } diff --git a/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java index 04a519dc5470..6099bb92e09e 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java @@ -58,4 +58,11 @@ public void testPartitionFieldIgnored() throws IOException final ShardSpec noneShardSpec = mapper.readValue(jsonStr, ShardSpec.class); noneShardSpec.equals(NoneShardSpec.instance()); } + + @Test + public void testCompatible() + { + final NoneShardSpec s1 = NoneShardSpec.instance(); + Assert.assertTrue(s1.isCompatible(NoneShardSpecFactory.instance().getShardSpecClass())); + } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 5b186c44ca0c..28a152ad062a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -38,10 +38,12 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +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.ResetDataSourceMetadataAction; +import org.apache.druid.indexing.common.actions.SegmentLockAquireAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -299,7 +301,24 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception final String topic = ioConfig.getStartSequenceNumbers().getStream(); // Start up, set up initial offsets. - final Object restoredMetadata = driver.startJob(); + final Object restoredMetadata = driver.startJob( + segmentId -> { + try { + return toolbox.getTaskActionClient().submit( + new SegmentLockAquireAction( + TaskLockType.EXCLUSIVE, + segmentId.getInterval(), + segmentId.getVersion(), + segmentId.getShardSpec().getPartitionNum(), + 1000L + ) + ).isOk(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ); if (restoredMetadata == null) { nextOffsets.putAll(ioConfig.getStartSequenceNumbers().getPartitionSequenceNumberMap()); } else { 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 6b9a20365b55..baf748ceb688 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 @@ -25,7 +25,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicate; 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; @@ -90,6 +89,7 @@ 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; @@ -176,6 +176,7 @@ 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; @@ -188,6 +189,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static org.apache.druid.query.QueryPlus.wrap; @@ -432,17 +434,18 @@ public void testRunAfterDataInserted() throws Exception 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 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) @@ -482,17 +485,18 @@ public void testRunBeforeDataInserted() throws Exception 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 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) @@ -574,14 +578,14 @@ public void testIncrementalHandOff() throws Exception 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)) @@ -590,15 +594,16 @@ public void testIncrementalHandOff() throws Exception ); // 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 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) @@ -721,21 +726,21 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception 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)) ), 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)) @@ -744,15 +749,16 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception ); // 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 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) @@ -830,9 +836,9 @@ public void testTimeBasedIncrementalHandOff() throws Exception 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)) @@ -841,8 +847,9 @@ public void testTimeBasedIncrementalHandOff() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + final List 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) @@ -993,17 +1000,18 @@ public void testRunWithMinimumMessageTime() throws Exception 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 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) @@ -1043,19 +1051,20 @@ public void testRunWithMaximumMessageTime() throws Exception 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 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) @@ -1103,16 +1112,17 @@ public void testRunWithTransformSpec() throws Exception 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 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) @@ -1147,7 +1157,7 @@ public void testRunOnNothing() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - Assert.assertEquals(ImmutableSet.of(), publishedDescriptors()); + Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); } @Test(timeout = 60_000L) @@ -1184,17 +1194,18 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception 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 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) @@ -1232,9 +1243,10 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio 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 publishedDescriptors = publishedDescriptors(); + assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)) @@ -1243,8 +1255,8 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio ); // 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) @@ -1285,7 +1297,7 @@ public void testReportParseExceptions() throws Exception 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())); } @@ -1329,11 +1341,11 @@ public void testMultipleParseExceptionsSuccess() throws Exception 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()) @@ -1407,7 +1419,7 @@ public void testMultipleParseExceptionsFailure() throws Exception 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(); @@ -1485,17 +1497,18 @@ public void testRunReplicas() throws Exception 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 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) @@ -1550,17 +1563,18 @@ public void testRunConflicting() throws Exception 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 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) @@ -1603,9 +1617,9 @@ public void testRunConflictingWithoutTransactions() throws Exception 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 @@ -1621,16 +1635,17 @@ public void testRunConflictingWithoutTransactions() throws Exception 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 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) @@ -1665,14 +1680,15 @@ public void testRunOneTaskTwoPartitions() throws Exception 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(isIncrementalHandoffSupported - ? ImmutableSet.of(desc1, desc2, desc4) - : ImmutableSet.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + SegmentDescriptor desc3 = sd("2011/P1D", 1); + SegmentDescriptor desc4 = sd("2012/P1D", 0); + final List publishedDescriptors = publishedDescriptors(); + assertEqualsExceptVersion(isIncrementalHandoffSupported + ? ImmutableList.of(desc1, desc2, desc4) + : ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)) @@ -1681,8 +1697,14 @@ public void testRunOneTaskTwoPartitions() throws Exception ); // 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", + isIncrementalHandoffSupported ? publishedDescriptors.get(2) : publishedDescriptors.get(3) + ) + ); // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically Assert.assertEquals( @@ -1690,8 +1712,11 @@ public void testRunOneTaskTwoPartitions() throws Exception ? ImmutableSet.of(ImmutableList.of("d", "e", "h")) : ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), isIncrementalHandoffSupported - ? ImmutableSet.of(readSegmentColumn("dim1", desc2)) - : ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) + ? ImmutableSet.of(readSegmentColumn("dim1", publishedDescriptors.get(1))) + : ImmutableSet.of( + readSegmentColumn("dim1", publishedDescriptors.get(1)), + readSegmentColumn("dim1", publishedDescriptors.get(2)) + ) ); } @@ -1746,10 +1771,10 @@ public void testRunTwoTasksTwoPartitions() throws Exception 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)) @@ -1758,9 +1783,10 @@ public void testRunTwoTasksTwoPartitions() throws Exception ); // 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 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) @@ -1846,17 +1872,18 @@ public void testRestore() throws Exception 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 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) @@ -1959,14 +1986,14 @@ public void testRestoreAfterPersistingSequences() throws Exception 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()) @@ -2049,17 +2076,18 @@ public void testRunWithPauseAndResume() throws Exception 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 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) @@ -2178,17 +2206,18 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception 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 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) @@ -2319,21 +2348,22 @@ public void testRunTransactionModeRollback() throws Exception 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 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) @@ -2635,7 +2665,7 @@ private void makeToolboxFactory() throws IOException derby.metadataTablesConfigSupplier().get(), derbyConnector ); - taskLockbox = new TaskLockbox(taskStorage); + taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( taskLockbox, taskStorage, @@ -2746,14 +2776,12 @@ private void destroyToolboxFactory() metadataStorageCoordinator = null; } - private Set publishedDescriptors() + private List 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) @@ -2853,10 +2881,33 @@ private static byte[] jb(String timestamp, String dim1, String dim2, String dimL } } - 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 descriptors1, List descriptors2) + { + Assert.assertEquals(descriptors1.size(), descriptors2.size()); + final Comparator 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 copy1 = new ArrayList<>(descriptors1); + final List 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/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 969cc3996368..c104d53ec0f2 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 @@ -31,7 +31,6 @@ import com.google.common.base.Predicate; 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; @@ -98,6 +97,7 @@ 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; @@ -173,6 +173,7 @@ 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; @@ -186,6 +187,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; import static org.easymock.EasyMock.anyLong; @@ -429,9 +431,9 @@ public void testRunAfterDataInserted() throws Exception 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(shardId1, "4")) @@ -440,8 +442,9 @@ public void testRunAfterDataInserted() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + final List 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) @@ -498,9 +501,9 @@ public void testRunBeforeDataInserted() throws Exception 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(shardId0, "1")) @@ -509,8 +512,9 @@ public void testRunBeforeDataInserted() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc2)); + final List 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) @@ -602,14 +606,14 @@ public void testIncrementalHandOff() throws Exception 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<>( @@ -621,15 +625,16 @@ public void testIncrementalHandOff() throws Exception ); // 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 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) @@ -752,25 +757,26 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception 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(shardId1, "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 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))); } @@ -829,17 +835,18 @@ public void testRunWithMinimumMessageTime() throws Exception 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(shardId1, "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 publishedDescriptors = publishedDescriptors(); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -898,10 +905,10 @@ public void testRunWithMaximumMessageTime() throws Exception 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(shardId1, "4"))), @@ -909,9 +916,10 @@ public void testRunWithMaximumMessageTime() throws Exception ); // 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 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))); } @@ -978,8 +986,8 @@ public void testRunWithTransformSpec() throws Exception 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(shardId1, "4"))), @@ -987,8 +995,9 @@ public void testRunWithTransformSpec() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); + final List publishedDescriptors = publishedDescriptors(); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(0))); + Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", publishedDescriptors.get(0))); } @@ -1044,7 +1053,7 @@ public void testRunOnSingletonRange() throws Exception 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()); } @@ -1100,9 +1109,9 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception 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(shardId1, "4")) @@ -1111,8 +1120,9 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + final List publishedDescriptors = publishedDescriptors(); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1169,9 +1179,9 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio 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(shardId1, "4")) @@ -1180,8 +1190,9 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + final List publishedDescriptors = publishedDescriptors(); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1238,7 +1249,7 @@ public void testReportParseExceptions() throws Exception 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())); } @@ -1301,11 +1312,11 @@ public void testMultipleParseExceptionsSuccess() throws Exception 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(shardId1, "12")) @@ -1398,7 +1409,7 @@ public void testMultipleParseExceptionsFailure() throws Exception 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(); @@ -1500,9 +1511,9 @@ public void testRunReplicas() throws Exception 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(shardId1, "4")) @@ -1511,8 +1522,9 @@ public void testRunReplicas() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + final List publishedDescriptors = publishedDescriptors(); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1592,17 +1604,18 @@ public void testRunConflicting() throws Exception 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(shardId1, "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 publishedDescriptors = publishedDescriptors(); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1669,9 +1682,9 @@ public void testRunConflictingWithoutTransactions() throws Exception 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 @@ -1689,16 +1702,17 @@ public void testRunConflictingWithoutTransactions() throws Exception 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 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))); } @@ -1761,10 +1775,10 @@ public void testRunOneTaskTwoPartitions() throws Exception 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(shardId1, "4", shardId0, "1")) @@ -1773,13 +1787,14 @@ public void testRunOneTaskTwoPartitions() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); + final List 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))) ); } @@ -1859,12 +1874,12 @@ public void testRunTwoTasksTwoPartitions() throws Exception 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(shardId1, "4", shardId0, "1")) @@ -1873,13 +1888,14 @@ public void testRunTwoTasksTwoPartitions() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + final List 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))); } @@ -1988,9 +2004,9 @@ public void testRestore() throws Exception 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(shardId1, "5"))), @@ -1998,8 +2014,9 @@ public void testRestore() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + final List 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) @@ -2127,13 +2144,13 @@ public void testRestoreAfterPersistingSequences() throws Exception 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(shardId1, "6")) @@ -2229,9 +2246,9 @@ public void testRunWithPauseAndResume() throws Exception 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, @@ -2241,8 +2258,9 @@ public void testRunWithPauseAndResume() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + final List 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) @@ -2310,17 +2328,18 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception 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(shardId1, "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 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) @@ -2458,16 +2477,16 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - final Set 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 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(shardId1, "9")) @@ -2767,7 +2786,7 @@ private void makeToolboxFactory() throws IOException derby.metadataTablesConfigSupplier().get(), derbyConnector ); - taskLockbox = new TaskLockbox(taskStorage); + taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( taskLockbox, taskStorage, @@ -2872,14 +2891,12 @@ private void destroyToolboxFactory() } - private Set publishedDescriptors() + private List 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) @@ -2987,10 +3004,33 @@ private static List jb( } } - 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 descriptors1, List descriptors2) + { + Assert.assertEquals(descriptors1.size(), descriptors2.size()); + final Comparator 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 copy1 = new ArrayList<>(descriptors1); + final List 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 0fb017e967d4..164c389f6db3 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 @@ -813,6 +813,7 @@ public void doRun() -1, -1 ); + final DataSegment segment = JobHelper.serializeOutIndex( segmentTemplate, context.getConfiguration(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java new file mode 100644 index 000000000000..bf71494d44cc --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java @@ -0,0 +1,26 @@ +/* + * 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; + +public enum LockGranularity +{ + 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 000000000000..ea06e63f3584 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/SegmentLock.java @@ -0,0 +1,238 @@ +/* + * 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, partitionId)?? + * TODO: is the above sentence true?? + */ +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("lockType") 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); + } + + @JsonProperty + @Override + public String getType() + { + return TYPE; + } + + @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 getLockType() + { + 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 TimeChunkLockRequest) { + // For different interval, all overlapping intervals cause conflict. + return dataSource.equals(request.getDataSource()) + && interval.overlaps(request.getInterval()); + } else if (request instanceof SpecificSegmentLockRequest) { + if (dataSource.equals(request.getDataSource()) + && 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 dataSource.equals(request.getDataSource()) + && interval.overlaps(request.getInterval()); + } + } else if (request instanceof LockRequestForNewSegment) { + // request for new segments doens't conflict with any locks because it allocates a new partitionId + return false; + } else { + throw new ISE("Unknown request type[%s]", request.getClass().getCanonicalName()); + } + } + + @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 865b053b95c4..df8881bc4f92 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,37 @@ /** * Represents a lock held by some task. Immutable. */ -public class TaskLock +@JsonTypeInfo(use = Id.NAME, property = "type", 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; - - 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 - @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; - } - - public TaskLock( - TaskLockType type, - String groupId, - String dataSource, - Interval interval, - String version, - int priority - ) - { - this(type, groupId, dataSource, interval, version, priority, false); - } - - public TaskLock revokedCopy() - { - return new TaskLock( - type, - groupId, - dataSource, - interval, - version, - priority, - true - ); - } - - @JsonProperty - public TaskLockType getType() - { - return type; - } - - @JsonProperty - public String getGroupId() - { - return groupId; - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public Interval getInterval() - { - return interval; - } - - @JsonProperty - public String getVersion() - { - return version; - } - - @JsonProperty + String getType(); + + TaskLock revokedCopy(); + + TaskLock withPriority(int priority); + + LockGranularity getGranularity(); + + TaskLockType getLockType(); + + String getGroupId(); + + String getDataSource(); + + Interval getInterval(); + + String getVersion(); + @Nullable - public Integer getPriority() - { - return priority; - } - - public int getNonNullPriority() - { - return Preconditions.checkNotNull(priority, "priority"); - } - - @JsonProperty - public boolean isRevoked() - { - return revoked; - } - - @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; - } - } - - @Override - public int hashCode() - { - return Objects.hashCode(type, 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(); - } + Integer getPriority(); + + int getNonNullPriority(); + + boolean isRevoked(); + + 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 e1f3f4e062dc..b51990cee03e 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 @@ 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/TimeChunkLock.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java new file mode 100644 index 000000000000..871e9d4f9bf0 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java @@ -0,0 +1,220 @@ +/* + * 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.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; + +public class TimeChunkLock implements TaskLock +{ + 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; + + @JsonCreator + public TimeChunkLock( + @JsonProperty("lockType") @Nullable TaskLockType lockType, // 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.lockType = lockType == null ? TaskLockType.EXCLUSIVE : 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.priority = priority; + this.revoked = revoked; + } + + @VisibleForTesting + public TimeChunkLock( + TaskLockType type, + String groupId, + String dataSource, + Interval interval, + String version, + int priority + ) + { + this(type, groupId, dataSource, interval, version, priority, false); + } + + @JsonProperty + @Override + public String getType() + { + return TYPE; + } + + @Override + public TaskLock revokedCopy() + { + return new TimeChunkLock( + lockType, + groupId, + dataSource, + interval, + version, + priority, + true + ); + } + + @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 getLockType() + { + 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() + { + return priority; + } + + @Override + public int getNonNullPriority() + { + return Preconditions.checkNotNull(priority, "priority"); + } + + @Override + @JsonProperty + public boolean isRevoked() + { + return revoked; + } + + @Override + public boolean conflict(LockRequest request) + { + return dataSource.equals(request.getDataSource()) + && interval.overlaps(request.getInterval()); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + 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.hash(lockType, groupId, dataSource, interval, version, priority, revoked); + } + + @Override + public String 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 97959d7be13a..71442b12bcb2 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,21 @@ 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.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; + public class LocalTaskActionClient implements TaskActionClient { + private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); + + private final ConcurrentMap, 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 +80,17 @@ public RetType submit(TaskAction taskAction) 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 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 6623483667e4..98a6499c2f8e 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,15 @@ 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.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 +36,13 @@ 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; @@ -53,6 +57,8 @@ * although you *do* have to release them yourself. *

* If this action cannot acquire an appropriate lock, or if it cannot expand an existing segment set, it returns null. + * + * TODO: must be used with segment locking */ public class SegmentAllocateAction implements TaskAction { @@ -68,7 +74,9 @@ public class SegmentAllocateAction implements TaskAction private final String sequenceName; private final String previousSegmentId; private final boolean skipSegmentLineageCheck; + private final ShardSpecFactory shardSpecFactory; + @JsonCreator public SegmentAllocateAction( @JsonProperty("dataSource") String dataSource, @JsonProperty("timestamp") DateTime timestamp, @@ -76,7 +84,8 @@ public SegmentAllocateAction( @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 ) { this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); @@ -89,6 +98,7 @@ public SegmentAllocateAction( this.sequenceName = Preconditions.checkNotNull(sequenceName, "sequenceName"); this.previousSegmentId = previousSegmentId; this.skipSegmentLineageCheck = skipSegmentLineageCheck; + this.shardSpecFactory = shardSpecFactory == null ? NumberedShardSpecFactory.instance() : shardSpecFactory; } @JsonProperty @@ -133,6 +143,12 @@ public boolean isSkipSegmentLineageCheck() return skipSegmentLineageCheck; } + @JsonProperty + public ShardSpecFactory getShardSpecFactory() + { + return shardSpecFactory; + } + @Override public TypeReference getReturnTypeReference() { @@ -162,18 +178,18 @@ public SegmentIdWithShardSpec perform( final Interval rowInterval = queryGranularity.bucket(timestamp); - final Set usedSegmentsForRow = ImmutableSet.copyOf( + final Set 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 +273,31 @@ private SegmentIdWithShardSpec tryAllocate( 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. + // TODO: don't use taskLockbox + final LockResult lockResult = toolbox.getTaskLockbox().tryLock( + task, + new LockRequestForNewSegment( + 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 - .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 +344,8 @@ public String toString() ", preferredSegmentGranularity=" + preferredSegmentGranularity + ", sequenceName='" + sequenceName + '\'' + ", previousSegmentId='" + previousSegmentId + '\'' + - ", skipSegmentLineageCheck='" + skipSegmentLineageCheck + '\'' + + ", skipSegmentLineageCheck=" + skipSegmentLineageCheck + + ", shardSpecFactory=" + shardSpecFactory + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java new file mode 100644 index 000000000000..b934664433db --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +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.TaskLockType; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.LockRequest; +import org.apache.druid.indexing.overlord.LockRequestForNewSegment; +import org.apache.druid.indexing.overlord.LockResult; +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.ShardSpecFactory; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +public class SegmentBulkAllocateAction implements TaskAction>> +{ + // interval -> # of segments to allocate + private final Map> allocateSpec; + private final String baseSequenceName; + + @JsonCreator + public SegmentBulkAllocateAction( + @JsonProperty("allocateSpec") Map> allocateSpec, + @JsonProperty("baseSequenceName") String baseSequenceName + ) + { + this.allocateSpec = allocateSpec; + this.baseSequenceName = baseSequenceName; + } + + @JsonProperty + public Map> getAllocateSpec() + { + return allocateSpec; + } + + @JsonProperty + public String getBaseSequenceName() + { + return baseSequenceName; + } + + @Override + public TypeReference>> getReturnTypeReference() + { + return new TypeReference>>() + { + }; + } + + @Override + public Map> perform(Task task, TaskActionToolbox toolbox) + { + final Map> segmentIds = new HashMap<>(allocateSpec.size()); + + for (Entry> 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 + ); + + for (int i = 0; i < numSegmentsToAllocate; i++) { + final String sequenceName = StringUtils.format("%s_%d", baseSequenceName, i); + final LockRequest lockRequest = new LockRequestForNewSegment( + TaskLockType.EXCLUSIVE, + task.getGroupId(), + task.getDataSource(), + interval, + shardSpecFactory, + task.getPriority(), + sequenceName, + null, + true + ); + + final LockResult lockResult = toolbox.getTaskLockbox().tryLock(task, lockRequest); + + if (lockResult.isRevoked()) { + // The lock was preempted by other tasks + throw new ISE("WTH? lock[%s] for new segment request is revoked?", lockResult.getTaskLock()); + } + + if (lockResult.isOk()) { + final SegmentIdWithShardSpec identifier = lockResult.getNewSegmentId(); + if (identifier != null) { + segmentIds.computeIfAbsent(interval, k -> new ArrayList<>()).add(identifier); + } else { + throw new ISE("Cannot allocate new pending segmentIds with request[%s]", lockRequest); + } + } else { + throw new ISE("Could not acquire lock with request[%s]", lockRequest); + } + } + } + + return segmentIds; + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return "SegmentBulkAllocateAction{" + + "allocateSpec=" + allocateSpec + + ", baseSequenceName='" + baseSequenceName + '\'' + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java new file mode 100644 index 000000000000..b9957090b590 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +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.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; + +public class SegmentLockAquireAction implements TaskAction +{ + private final TaskLockType lockType; + private final Interval interval; + private final String version; + private final int partitionId; + private final long timeoutMs; + + @JsonCreator + public SegmentLockAquireAction( + @JsonProperty("lockType") TaskLockType lockType, + @JsonProperty("interval") Interval interval, + @JsonProperty("version") String version, + @JsonProperty("partitionId") int partitionId, + @JsonProperty("timeoutMs") long timeoutMs + ) + { + 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 + public TaskLockType getLockType() + { + return lockType; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public String getVersion() + { + return version; + } + + @JsonProperty + public int getPartitionId() + { + return partitionId; + } + + @JsonProperty + public long getTimeoutMs() + { + return timeoutMs; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public LockResult perform(Task task, TaskActionToolbox toolbox) + { + try { + 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); + } + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return "SegmentLockAquireAction{" + + "lockType=" + lockType + + ", interval=" + interval + + ", version='" + version + '\'' + + ", partitionId=" + partitionId + + ", timeoutMs=" + timeoutMs + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java new file mode 100644 index 000000000000..f1e20dd19ca3 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +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.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 java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +public class SegmentLockTryAcquireAction implements TaskAction> +{ + private final TaskLockType type; + private final Interval interval; + private final String version; + private final Set partitionIds; + + @JsonCreator + public SegmentLockTryAcquireAction( + @JsonProperty("lockType") TaskLockType type, + @JsonProperty("interval") Interval interval, + @JsonProperty("version") String version, + @JsonProperty("partitionIds") Set partitionIds // TODO: IntSet + ) + { + Preconditions.checkState(partitionIds != null && !partitionIds.isEmpty(), "partitionIds is empty"); + this.type = Preconditions.checkNotNull(type, "type"); + this.interval = Preconditions.checkNotNull(interval, "interval"); + this.version = Preconditions.checkNotNull(version, "version"); + this.partitionIds = partitionIds; + } + + @JsonProperty("lockType") + public TaskLockType getType() + { + return type; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public String getVersion() + { + return version; + } + + @JsonProperty + public Set getPartitionIds() + { + return partitionIds; + } + + @Override + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() + { + }; + } + + @Override + public List perform(Task task, TaskActionToolbox toolbox) + { + return partitionIds.stream() + .map(partitionId -> toolbox.getTaskLockbox().tryLock( + task, + new SpecificSegmentLockRequest(type, task, interval, version, partitionId) + )) + .collect(Collectors.toList()); + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return "SegmentLockTryAcquireAction{" + + "type=" + type + + ", interval=" + interval + + ", 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 446ee1110038..0a8500261f43 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 @@ -34,9 +34,11 @@ import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; public class SegmentMetadataUpdateAction implements TaskAction { @@ -68,12 +70,16 @@ public Void perform(Task task, TaskActionToolbox toolbox) { TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); + final Map> intervalToPartitionIds = new HashMap<>(); + for (DataSegment segment : segments) { + intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) + .add(segment.getShardSpec().getPartitionNum()); + } try { toolbox.getTaskLockbox().doInCriticalSection( task, - intervals, + intervalToPartitionIds, 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 f81028e1aad2..d3dd0ddd8bd9 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 @@ -34,9 +34,11 @@ import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; public class SegmentNukeAction implements TaskAction { @@ -70,12 +72,16 @@ public Void perform(Task task, TaskActionToolbox toolbox) { TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); + final Map> intervalToPartitionIds = new HashMap<>(); + for (DataSegment segment : segments) { + intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) + .add(segment.getShardSpec().getPartitionNum()); + } try { toolbox.getTaskLockbox().doInCriticalSection( task, - intervals, + intervalToPartitionIds, 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 ca595a0505a1..ebc4acb14bed 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 @@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; 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.Task; import org.apache.druid.indexing.overlord.CriticalAction; @@ -31,9 +33,14 @@ 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.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.Set; -import java.util.stream.Collectors; /** * Insert segments into metadata storage. The segment versions must all be less than or equal to a lock held by @@ -45,14 +52,11 @@ */ public class SegmentTransactionalInsertAction implements TaskAction { - private final Set segments; private final DataSourceMetadata startMetadata; private final DataSourceMetadata endMetadata; - public SegmentTransactionalInsertAction( - Set segments - ) + public SegmentTransactionalInsertAction(Set segments) { this(segments, null, null); } @@ -102,13 +106,20 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { + // TODO: move this to lock checking in critical section TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + final Map> intervalToPartitionIds = new HashMap<>(); + for (DataSegment segment : segments) { + intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) + .add(segment.getShardSpec().getPartitionNum()); + } + final SegmentPublishResult retVal; try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, - segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), + intervalToPartitionIds, CriticalAction.builder() .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( @@ -130,6 +141,15 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throw new RuntimeException(e); } + final List locks = toolbox.getTaskLockbox().findLocksForTask(task); + if (locks.get(0).getGranularity() == LockGranularity.SEGMENT) { + for (Entry> entry : intervalToPartitionIds.entrySet()) { + final Interval interval = entry.getKey(); + final List partitionIds = entry.getValue(); + partitionIds.forEach(partitionId -> toolbox.getTaskLockbox().unlock(task, interval, partitionId)); + } + } + // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); 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 206fd5fde7be..efb1c82da090 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,9 @@ @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 = "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 index 6db8d8e2ab5b..c4429c587975 100644 --- 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 @@ -20,13 +20,17 @@ 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.List; import java.util.Map.Entry; @@ -64,31 +68,53 @@ static boolean isLockCoversSegments( // 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 taskLockMap = getTaskLockMap(taskLockbox, task); + final NavigableMap> taskLockMap = getTaskLockMap(taskLockbox, task); if (taskLockMap.isEmpty()) { return false; } + return isLockCoversSegments(taskLockMap, segments); + } + + public static boolean isLockCoversSegments( + NavigableMap> taskLockMap, + Collection segments + ) + { return segments.stream().allMatch( segment -> { - final Entry entry = taskLockMap.floorEntry(segment.getInterval().getStart()); + final Entry> 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; + final List 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(); + } + } + ); } ); } - private static NavigableMap getTaskLockMap(TaskLockbox taskLockbox, Task task) + private static NavigableMap> getTaskLockMap(TaskLockbox taskLockbox, Task task) { final List taskLocks = taskLockbox.findLocksForTask(task); - final NavigableMap taskLockMap = new TreeMap<>(); - taskLocks.forEach(taskLock -> taskLockMap.put(taskLock.getInterval().getStart(), taskLock)); + final NavigableMap> 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 82% 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 0b2e49d6e501..1441fc1b6c70 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,12 @@ 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 +public class TimeChunkLockAcquireAction implements TaskAction { private final TaskLockType type; @@ -43,7 +44,7 @@ public class LockAcquireAction implements TaskAction 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 +85,11 @@ public TypeReference getReturnTypeReference() 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 +106,7 @@ public boolean isAudited() @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 85% 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 f24266ba4b9a..e4c54d5050b9 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,12 @@ 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 +public class TimeChunkLockTryAcquireAction implements TaskAction { @JsonIgnore private final TaskLockType type; @@ -40,7 +41,7 @@ public class LockTryAcquireAction implements TaskAction private final Interval interval; @JsonCreator - public LockTryAcquireAction( + public TimeChunkLockTryAcquireAction( @JsonProperty("lockType") @Nullable TaskLockType type, // nullable for backward compatibility @JsonProperty("interval") Interval interval ) @@ -72,7 +73,10 @@ public TypeReference getReturnTypeReference() @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 +89,8 @@ public boolean isAudited() @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 6e0e1d861d16..edc74ab69276 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 Plumber findPlumber( 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/AbstractFixedIntervalTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java index 6959677e7812..90abc3b6b79e 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 @@ -22,11 +22,10 @@ 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.joda.time.Interval; +import java.util.Collections; import java.util.Map; public abstract class AbstractFixedIntervalTask extends AbstractTask @@ -72,7 +71,7 @@ protected AbstractFixedIntervalTask( @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - return taskActionClient.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) != null; + return tryLockWithIntervals(taskActionClient, Collections.singletonList(interval)); } @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 27e585e326fd..a73eb254cfa6 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 @@ -24,22 +24,41 @@ import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; +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 javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; public abstract class AbstractTask implements Task { @@ -59,6 +78,41 @@ public abstract class AbstractTask implements Task private final Map context; + @Nullable + private Map overwritingSegmentMetas; + + @Nullable + private Boolean changeSegmentGranularity; + + public static class OverwritingSegmentMeta + { + private final int startRootPartitionId; + private final int endRootPartitionId; + private final short minorVersionForNewSegments; + + private OverwritingSegmentMeta(int startRootPartitionId, int endRootPartitionId, short minorVersionForNewSegments) + { + this.startRootPartitionId = startRootPartitionId; + this.endRootPartitionId = endRootPartitionId; + this.minorVersionForNewSegments = minorVersionForNewSegments; + } + + public int getStartRootPartitionId() + { + return startRootPartitionId; + } + + public int getEndRootPartitionId() + { + return endRootPartitionId; + } + + public short getMinorVersionForNewSegments() + { + return minorVersionForNewSegments; + } + } + protected AbstractTask(String id, String dataSource, Map context) { this(id, null, null, dataSource, context); @@ -242,4 +296,211 @@ public Map getContext() { return context; } + + // TODO: remove this and check by findInputSegments returns empty? + public abstract boolean requireLockInputSegments(); + + public abstract List findInputSegments(TaskActionClient taskActionClient, List intervals) throws IOException; + + public abstract boolean changeSegmentGranularity(List intervalOfExistingSegments); + + /** + * Returns the segmentGranularity for the given interval. Usually tasks are supposed to return its segmentGranularity + * if exists. The compactionTask can return different segmentGranularity depending on its configuration and the input + * interval. + * + * @return segmentGranularity or null if it doesn't support it. + */ + @Nullable + public abstract Granularity getSegmentGranularity(Interval interval); + + protected boolean tryLockWithIntervals(TaskActionClient client, Set intervals) + throws IOException + { + return tryLockWithIntervals(client, new ArrayList<>(intervals)); + } + + protected boolean tryLockWithIntervals(TaskActionClient client, List intervals) + throws IOException + { + if (requireLockInputSegments()) { + final List intervalsToFindInput = new ArrayList<>(intervals); + if (overwritingSegmentMetas != null) { + intervalsToFindInput.removeAll(overwritingSegmentMetas.keySet()); + } + + // TODO: check changeSegmentGranularity and get timeChunkLock here + + // TODO: race - a new segment can be added after findInputSegments. change to lockAllSegmentsInIntervals + if (!intervalsToFindInput.isEmpty()) { + return tryLockWithSegments(client, findInputSegments(client, intervalsToFindInput)); + } else { + return true; + } + } else { + changeSegmentGranularity = false; + overwritingSegmentMetas = Collections.emptyMap(); + return true; + } + } + + protected boolean tryLockWithSegments(TaskActionClient client, List segments) throws IOException + { + if (requireLockInputSegments()) { + if (segments.isEmpty()) { + changeSegmentGranularity = false; + overwritingSegmentMetas = Collections.emptyMap(); + return true; + } + + // Create a timeline to find latest segments only + final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); + final TimelineLookup timeline = VersionedIntervalTimeline.forSegments(segments); + final List visibleSegments = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) + .stream() + .map(TimelineObjectHolder::getObject) + .flatMap(partitionHolder -> StreamSupport.stream( + partitionHolder.spliterator(), + false + )) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); + + changeSegmentGranularity = changeSegmentGranularity(intervals); + if (changeSegmentGranularity) { + overwritingSegmentMetas = Collections.emptyMap(); + // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined + final Set uniqueIntervals = new HashSet<>(); + for (Interval interval : JodaUtils.condenseIntervals(intervals)) { + final Granularity segmentGranularity = getSegmentGranularity(interval); + 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; + } else { + final Map> intervalToSegments = new HashMap<>(); + for (DataSegment segment : segments) { + intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); + } + intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); + for (Entry> entry : intervalToSegments.entrySet()) { + final Interval interval = entry.getKey(); + final Set partitionIds = entry.getValue().stream() + .map(s -> s.getShardSpec().getPartitionNum()) + .collect(Collectors.toSet()); + final List lockResults = client.submit( + new SegmentLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval, visibleSegments.get(0).getVersion(), partitionIds) + ); + if (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk())) { + return false; + } + } + return true; + } + } else { + changeSegmentGranularity = false; + overwritingSegmentMetas = Collections.emptyMap(); + return true; + } + } + + private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments) + { + if (inputSegments.isEmpty()) { + return; + } + + Preconditions.checkArgument( + inputSegments.stream().allMatch(segment -> segment.getInterval().equals(inputSegments.get(0).getInterval())) + ); + final Interval interval = inputSegments.get(0).getInterval(); + + inputSegments.sort((s1, s2) -> { + if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) { + return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId()); + } else { + return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId()); + } + }); + + short atomicUpdateGroupSize = 1; + // sanity check + for (int i = 0; i < inputSegments.size() - 1; i++) { + final DataSegment curSegment = inputSegments.get(i); + final DataSegment nextSegment = inputSegments.get(i + 1); + if (curSegment.getStartRootPartitionId() == nextSegment.getStartRootPartitionId() + && curSegment.getEndRootPartitionId() == nextSegment.getEndRootPartitionId()) { + // Input segments should have the same or consecutive rootPartition range + if (curSegment.getMinorVersion() != nextSegment.getMinorVersion() + || curSegment.getAtomicUpdateGroupSize() != nextSegment.getAtomicUpdateGroupSize()) { + throw new ISE( + "segment[%s] and segment[%s] have the same rootPartitionRange, but different minorVersion or atomicUpdateGroupSize", + curSegment, + nextSegment + ); + } + atomicUpdateGroupSize++; + } else { + if (curSegment.getEndRootPartitionId() != nextSegment.getStartRootPartitionId()) { + throw new ISE("Can't compact segments of non-consecutive rootPartition range"); + } + if (atomicUpdateGroupSize != curSegment.getAtomicUpdateGroupSize()) { + throw new ISE("All atomicUpdateGroup must be compacted together"); + } + atomicUpdateGroupSize = 1; + } + } + + final short prevMaxMinorVersion = (short) inputSegments + .stream() + .mapToInt(DataSegment::getMinorVersion) + .max() + .orElseThrow(() -> new ISE("Empty inputSegments")); + + if (overwritingSegmentMetas == null) { + overwritingSegmentMetas = new HashMap<>(); + } + overwritingSegmentMetas.put( + interval, + new OverwritingSegmentMeta( + inputSegments.get(0).getStartRootPartitionId(), + inputSegments.get(inputSegments.size() - 1).getEndRootPartitionId(), + (short) (prevMaxMinorVersion + 1) + ) + ); + } + + protected boolean isChangeSegmentGranularity() + { + return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); + } + + public Map getAllOverwritingSegmentMeta() + { + Preconditions.checkNotNull(overwritingSegmentMetas, "overwritingSegmentMetas is not initialized"); + return overwritingSegmentMetas; + } + + @Nullable + public OverwritingSegmentMeta getOverwritingSegmentMeta(Interval interval) + { + Preconditions.checkNotNull(overwritingSegmentMetas, "overwritingSegmentMetas is not initialized"); + return overwritingSegmentMetas.get(interval); + } + + public boolean isOverwriteMode() + { + Preconditions.checkNotNull(overwritingSegmentMetas, "overwritingSegmentMetas is not initialized"); + return !overwritingSegmentMetas.isEmpty(); + } } 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 19a256883edd..eea46ea9bd3d 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,10 +44,12 @@ 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.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.SegmentLockAquireAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; @@ -60,6 +62,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.ListenableFutures; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -84,8 +87,12 @@ 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.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.utils.CircularBuffer; +import org.joda.time.Interval; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.GET; import javax.ws.rs.Path; @@ -94,6 +101,7 @@ 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; @@ -274,7 +282,24 @@ public TaskStatus run(final TaskToolbox toolbox) toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); - driver.startJob(); + driver.startJob( + segmentId -> { + try { + return toolbox.getTaskActionClient().submit( + new SegmentLockAquireAction( + TaskLockType.EXCLUSIVE, + segmentId.getInterval(), + segmentId.getVersion(), + segmentId.getShardSpec().getPartitionNum(), + 1000L + ) + ).isOk(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ); // Set up metrics emission toolbox.getMonitorScheduler().addMonitor(metricsMonitor); @@ -395,6 +420,31 @@ public TaskStatus run(final TaskToolbox toolbox) return TaskStatus.success(getId()); } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public boolean canRestore() { @@ -726,7 +776,8 @@ private static StreamAppenderatorDriver newDriver( schema.getGranularitySpec().getSegmentGranularity(), sequenceName, previousSegmentId, - skipSegmentLineageCheck + skipSegmentLineageCheck, + NumberedShardSpecFactory.instance() ) ), toolbox.getSegmentHandoffNotifierFactory(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java index 01d2448a88db..b4e3dd55d7b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java @@ -27,11 +27,15 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -105,4 +109,29 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } + + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } } 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 000000000000..5bef3db5c0fb --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.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.indexing.common.task; + +import com.google.common.base.Preconditions; +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.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class CachingLocalSegmentAllocator extends CachingSegmentAllocator +{ + private final String dataSource; + + private final Map intervalToVersion; + + public CachingLocalSegmentAllocator( + TaskToolbox toolbox, + String taskId, + String dataSource, + Map> allocateSpec + ) throws IOException + { + // This segment allocator doesn't need inputPartitionIds because the newly created segments don't have to store + // direcOvershadowingSegments + super(toolbox, taskId, allocateSpec); + this.dataSource = dataSource; + + intervalToVersion = toolbox.getTaskActionClient() + .submit(new LockListAction()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + } + + @Override + Map> getIntervalToSegmentIds() + { + final Map> allocateSpec = getAllocateSpec(); + final Map> intervalToSegmentIds = new HashMap<>(allocateSpec.size()); + for (Entry> entry : allocateSpec.entrySet()) { + final Interval interval = entry.getKey(); + final ShardSpecFactory shardSpecFactory = entry.getValue().lhs; + final int numSegmentsToAllocate = Preconditions.checkNotNull( + entry.getValue().rhs, + "numSegmentsToAllocate for interval[%s]", + interval + ); + + intervalToSegmentIds.put( + interval, + IntStream.range(0, numSegmentsToAllocate) + .mapToObj(i -> new SegmentIdWithShardSpec( + dataSource, + interval, + findVersion(interval), + shardSpecFactory.create(getToolbox().getObjectMapper(), i) + )) + .collect(Collectors.toList()) + ); + } + return intervalToSegmentIds; + } + + private String findVersion(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)); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java new file mode 100644 index 000000000000..cff358c49788 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.SegmentBulkAllocateAction; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class CachingRemoteSegmentAllocator extends CachingSegmentAllocator +{ + public CachingRemoteSegmentAllocator( + TaskToolbox toolbox, + String taskId, + Map> allocateSpec + ) throws IOException + { + super(toolbox, taskId, allocateSpec); + } + + @Override + Map> getIntervalToSegmentIds() throws IOException + { + return getToolbox().getTaskActionClient().submit( + new SegmentBulkAllocateAction( + getAllocateSpec(), + getTaskId() + ) + ); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java new file mode 100644 index 000000000000..05a96f5a3849 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs; +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; + +// TODO: caching?? +public abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocator +{ + private final TaskToolbox toolbox; + private final String taskId; + private final Map> allocateSpec; + @Nullable + private final ShardSpecs shardSpecs; + + // sequenceName -> segmentId + private final Map sequenceNameToSegmentId; + + public CachingSegmentAllocator( + TaskToolbox toolbox, + String taskId, + Map> allocateSpec + ) throws IOException + { + this.toolbox = toolbox; + this.taskId = taskId; + this.allocateSpec = allocateSpec; + this.sequenceNameToSegmentId = new HashMap<>(); + + final Map> intervalToIds = getIntervalToSegmentIds(); + final Map> shardSpecMap = new HashMap<>(); + + for (Map.Entry> entry : intervalToIds.entrySet()) { + final Interval interval = entry.getKey(); + final List idsPerInterval = intervalToIds.get(interval); + + for (SegmentIdWithShardSpec segmentIdentifier : idsPerInterval) { + shardSpecMap.computeIfAbsent(interval, k -> new ArrayList<>()).add(segmentIdentifier.getShardSpec()); + // The shardSpecs for partitinoing and publishing can be different if isExtendableShardSpecs = true. + sequenceNameToSegmentId.put(getSequenceName(interval, segmentIdentifier.getShardSpec()), segmentIdentifier); + } + } + shardSpecs = new ShardSpecs(shardSpecMap); + } + + abstract Map> getIntervalToSegmentIds() throws IOException; + + TaskToolbox getToolbox() + { + return toolbox; + } + + String getTaskId() + { + return taskId; + } + + Map> 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) + { + return getSequenceName(interval, shardSpecs.getShardSpec(interval, inputRow)); + } + + /** + * Create a sequence name from the given shardSpec and interval. The shardSpec must be the original one before calling + * {@link #makeShardSpec(ShardSpec, int)} to apply the proper partitioning. + * + * 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 156d0f6896ba..b93104769fe1 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 @@ -28,6 +28,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.data.input.impl.DimensionSchema; @@ -96,9 +97,7 @@ 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; @@ -240,8 +239,8 @@ public Boolean isKeepSegmentGranularity() return keepSegmentGranularity; } - @JsonProperty - public Granularity getSegmentGranularity() + @JsonProperty("segmentGranularity") + public Granularity getInputSegmentGranularity() { return segmentGranularity; } @@ -281,16 +280,54 @@ SegmentProvider getSegmentProvider() @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - intervals.add(segmentProvider.interval); - return IndexTask.isReady(taskActionClient, intervals); + final List segments = segmentProvider.checkAndGetSegments(taskActionClient); + return tryLockWithSegments(taskActionClient, segments); + } + + @Override + public boolean requireLockInputSegments() + { + return true; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + throws IOException + { + return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return (keepSegmentGranularity != null && !keepSegmentGranularity) + || (segmentGranularity != null); // TODO: check segmentGranularity is different + } + + @Override + public Granularity getSegmentGranularity(Interval interval) + { + if (segmentGranularity == null) { + if (keepSegmentGranularity != null && !keepSegmentGranularity) { + return Granularities.ALL; + } else { + return GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); + } + } else { + if (keepSegmentGranularity != null && keepSegmentGranularity) { + // error + throw new ISE("segmentGranularity[%s] and keepSegmentGranularity can't be used together", segmentGranularity); + } else { + return segmentGranularity; + } + } } @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { if (indexTaskSpecs == null) { - indexTaskSpecs = createIngestionSchema( + final List ingestionSpecs = createIngestionSchema( toolbox, segmentProvider, partitionConfigurationManager, @@ -302,19 +339,21 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception 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()) { @@ -347,6 +386,11 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } } + private String createIndexTaskSpecId(int i) + { + return StringUtils.format("%s_%d", getId(), i); + } + /** * Generate {@link IndexIngestionSpec} from input segments. * @@ -379,6 +423,7 @@ static List createIngestionSchema( } // find metadata for interval + // queryableIndexAndSegments is sorted by the interval of the dataSegment final List> queryableIndexAndSegments = loadSegments( timelineSegments, segmentFileMap, @@ -394,7 +439,6 @@ static List createIngestionSchema( // all granularity final DataSchema dataSchema = createDataSchema( segmentProvider.dataSource, - segmentProvider.interval, queryableIndexAndSegments, dimensionsSpec, metricsSpec, @@ -408,7 +452,7 @@ static List createIngestionSchema( createIoConfig( toolbox, dataSchema, - segmentProvider.interval, + Iterables.getOnlyElement(dataSchema.getGranularitySpec().inputIntervals()), coordinatorClient, segmentLoaderFactory, retryPolicyFactory @@ -433,7 +477,6 @@ static List createIngestionSchema( final List> segmentsToCompact = entry.getValue(); final DataSchema dataSchema = createDataSchema( segmentProvider.dataSource, - interval, segmentsToCompact, dimensionsSpec, metricsSpec, @@ -467,7 +510,6 @@ static List createIngestionSchema( // given segment granularity final DataSchema dataSchema = createDataSchema( segmentProvider.dataSource, - segmentProvider.interval, queryableIndexAndSegments, dimensionsSpec, metricsSpec, @@ -481,7 +523,7 @@ static List createIngestionSchema( createIoConfig( toolbox, dataSchema, - segmentProvider.interval, + Iterables.getOnlyElement(dataSchema.getGranularitySpec().inputIntervals()), coordinatorClient, segmentLoaderFactory, retryPolicyFactory @@ -526,7 +568,7 @@ private static Pair, List usedSegments = segmentProvider.checkAndGetSegments(toolbox); + final List usedSegments = segmentProvider.checkAndGetSegments(toolbox.getTaskActionClient()); final Map segmentFileMap = toolbox.fetchSegments(usedSegments); final List> timelineSegments = VersionedIntervalTimeline .forSegments(usedSegments) @@ -536,7 +578,6 @@ private static Pair, List> queryableIndexAndSegments, @Nullable DimensionsSpec dimensionsSpec, @Nullable AggregatorFactory[] metricsSpec, @@ -560,6 +601,10 @@ private static DataSchema createDataSchema( 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, @@ -745,6 +790,7 @@ static class SegmentProvider { private final String dataSource; private final Interval interval; + @Nullable private final List segments; SegmentProvider(String dataSource, Interval interval) @@ -762,21 +808,22 @@ static class SegmentProvider 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 getSegments() { return segments; } - List checkAndGetSegments(TaskToolbox toolbox) throws IOException + List checkAndGetSegments(TaskActionClient actionClient) throws IOException { - final List usedSegments = toolbox.getTaskActionClient().submit( + final List usedSegments = actionClient.submit( new SegmentListUsedAction(dataSource, interval, null) ); final TimelineLookup 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 5b67c259019e..eb3b4ba1f18a 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,15 +45,16 @@ 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.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; @@ -64,6 +65,7 @@ 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; @@ -198,12 +200,38 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception 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 requireLockInputSegments() + { + throw new UnsupportedOperationException(); + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + final Granularity segmentGranularity = spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); + return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); + } + @JsonProperty("spec") public HadoopIngestionSpec getSpec() { @@ -345,7 +373,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception // 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/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 5161f9bcba93..3cf0f6706b4d 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.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.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.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.SegmentListUsedAction; 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.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; @@ -78,7 +76,6 @@ 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,11 +85,14 @@ 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.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.apache.druid.utils.CircularBuffer; import org.codehaus.plexus.util.FileUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -107,22 +107,19 @@ import javax.ws.rs.core.Response; import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; 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 { @@ -256,13 +253,41 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception .bucketIntervals(); if (intervals.isPresent()) { - return isReady(taskActionClient, intervals.get()); + return tryLockIfNecessary(taskActionClient, intervals.get()); } else { return true; } } - static boolean isReady(TaskActionClient actionClient, SortedSet intervals) throws IOException + @Override + public boolean requireLockInputSegments() + { + return isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig) + || !ingestionSchema.ioConfig.isAppendToExisting(); + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + throws IOException + { + return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return ingestionSchema.dataSchema.getGranularitySpec().getSegmentGranularity(); + } + + private boolean tryLockIfNecessary(TaskActionClient actionClient, Collection intervals) throws IOException { // Sanity check preventing empty intervals (which cannot be locked, and don't make sense anyway). for (Interval interval : intervals) { @@ -271,16 +296,7 @@ static boolean isReady(TaskActionClient actionClient, SortedSet interv } } - final List locks = getTaskLocks(actionClient); - if (locks.size() == 0) { - try { - Tasks.tryAcquireExclusiveLocks(actionClient, intervals); - } - catch (Exception e) { - return false; - } - } - return true; + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals)); } @GET @@ -434,32 +450,33 @@ public TaskStatus run(final TaskToolbox toolbox) 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); + // Spec for segment allocation. This is used only for perfect rollup mode. + // See createSegmentAllocator(). + final Map> allocateSpec = determineShardSpecs( + toolbox, + firehoseFactory, + firehoseTempDir, + maxRowsPerSegment + ); + + final Set allocateIntervals = allocateSpec.keySet(); + // get locks for found shardSpec intervals + if (!tryLockIfNecessary(toolbox.getTaskActionClient(), allocateIntervals)) { + throw new ISE("Failed to get a lock for segments"); + } + final DataSchema dataSchema; - final Map versions; if (determineIntervals) { - final SortedSet intervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); - intervals.addAll(shardSpecs.getIntervals()); - final Map locks = Tasks.tryAcquireExclusiveLocks( - toolbox.getTaskActionClient(), - intervals - ); - versions = locks.entrySet().stream() - .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); + if (!tryLockWithIntervals(toolbox.getTaskActionClient(), allocateIntervals)) { + throw new ISE("Failed to get locks for intervals[%s]", allocateIntervals); + } dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( ingestionSchema.getDataSchema() .getGranularitySpec() - .withIntervals( - JodaUtils.condenseIntervals( - shardSpecs.getIntervals() - ) - ) + .withIntervals(JodaUtils.condenseIntervals(allocateIntervals)) ); } else { - versions = getTaskLocks(toolbox.getTaskActionClient()) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); dataSchema = ingestionSchema.getDataSchema(); } @@ -467,8 +484,7 @@ public TaskStatus run(final TaskToolbox toolbox) return generateAndPublishSegments( toolbox, dataSchema, - shardSpecs, - versions, + allocateSpec, firehoseFactory, firehoseTempDir, maxRowsPerSegment, @@ -539,13 +555,9 @@ private Map getTaskCompletionRowStats() return metrics; } - private static String findVersion(Map versions, Interval interval) + public interface SequenceNameGenerator { - return versions.entrySet().stream() - .filter(entry -> entry.getKey().contains(interval)) - .map(Entry::getValue) - .findFirst() - .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); + String getSequenceName(Interval interval, InputRow inputRow); } private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) @@ -572,7 +584,7 @@ private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningCon * * @return generated {@link ShardSpecs} representing a map of intervals and corresponding shard specs */ - private ShardSpecs determineShardSpecs( + private Map> determineShardSpecs( final TaskToolbox toolbox, final FirehoseFactory firehoseFactory, final File firehoseTempDir, @@ -595,12 +607,7 @@ private ShardSpecs determineShardSpecs( // if we were given number of shards per interval and the intervals, we don't need to scan the data if (!determineNumPartitions && !determineIntervals) { log.info("Skipping determine partition scan"); - return createShardSpecWithoutInputScan( - jsonMapper, - granularitySpec, - ioConfig, - tuningConfig - ); + return createShardSpecWithoutInputScan(granularitySpec, ioConfig, tuningConfig); } else { // determine intervals containing data and prime HLL collectors return createShardSpecsFromInput( @@ -617,43 +624,31 @@ private ShardSpecs determineShardSpecs( } } - private static ShardSpecs createShardSpecWithoutInputScan( - ObjectMapper jsonMapper, + private Map> createShardSpecWithoutInputScan( GranularitySpec granularitySpec, IndexIOConfig ioConfig, IndexTuningConfig tuningConfig ) { - final Map> shardSpecs = new HashMap<>(); + final Map> allocateSpec = new HashMap<>(); final SortedSet intervals = granularitySpec.bucketIntervals().get(); if (isGuaranteedRollup(ioConfig, tuningConfig)) { // Overwrite mode, guaranteed rollup: shardSpecs must be known in advance. final int numShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); - for (Interval interval : intervals) { - final List intervalShardSpecs = IntStream.range(0, numShards) - .mapToObj( - shardId -> new HashBasedNumberedShardSpec( - shardId, - numShards, - tuningConfig.partitionDimensions, - jsonMapper - ) - ) - .collect(Collectors.toList()); - shardSpecs.put(interval, intervalShardSpecs); + allocateSpec.put(interval, createShardSpecFactoryForGuaranteedRollup(numShards, tuningConfig.partitionDimensions)); } } else { for (Interval interval : intervals) { - shardSpecs.put(interval, ImmutableList.of()); + allocateSpec.put(interval, null); } } - return new ShardSpecs(shardSpecs); + return allocateSpec; } - private ShardSpecs createShardSpecsFromInput( + private Map> createShardSpecsFromInput( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, FirehoseFactory firehoseFactory, @@ -678,7 +673,7 @@ private ShardSpecs createShardSpecsFromInput( determineNumPartitions ); - final Map> intervalToShardSpecs = new HashMap<>(); + final Map> allocateSpecs = new HashMap<>(); final int defaultNumShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); for (final Map.Entry> entry : hllCollectors.entrySet()) { final Interval interval = entry.getKey(); @@ -697,25 +692,47 @@ private ShardSpecs createShardSpecsFromInput( } if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) { - // Overwrite mode, guaranteed rollup: shardSpecs must be known in advance. - final List intervalShardSpecs = IntStream.range(0, numShards) - .mapToObj( - shardId -> new HashBasedNumberedShardSpec( - shardId, - numShards, - tuningConfig.partitionDimensions, - jsonMapper - ) - ).collect(Collectors.toList()); - - intervalToShardSpecs.put(interval, intervalShardSpecs); + // Overwrite mode, guaranteed rollup: # of shards must be known in advance. + allocateSpecs.put(interval, createShardSpecFactoryForGuaranteedRollup(numShards, tuningConfig.partitionDimensions)); } else { - intervalToShardSpecs.put(interval, ImmutableList.of()); + allocateSpecs.put(interval, null); } } log.info("Found intervals and shardSpecs in %,dms", System.currentTimeMillis() - determineShardSpecsStartMillis); - return new ShardSpecs(intervalToShardSpecs); + return allocateSpecs; + } + + private Pair createShardSpecFactoryForGuaranteedRollup( + int numShards, + @Nullable List partitionDimensions + ) + { + return Pair.of(new HashBasedNumberedShardSpecFactory(partitionDimensions, numShards), numShards); + } + + private Pair createShardSpecFactoryForBestEffortRollup( + boolean overwrite, + Interval interval + ) + { + if (overwrite && !isChangeSegmentGranularity()) { + final OverwritingSegmentMeta overwritingSegmentMeta = Preconditions.checkNotNull( + getOverwritingSegmentMeta(interval), + "Can't find overwritingSegmentMeta for interval[%s]", + interval + ); + return Pair.of( + new NumberedOverwritingShardSpecFactory( + overwritingSegmentMeta.getStartRootPartitionId(), + overwritingSegmentMeta.getEndRootPartitionId(), + overwritingSegmentMeta.getMinorVersionForNewSegments() + ), + null + ); + } else { + return Pair.of(NumberedShardSpecFactory.instance(), null); + } } private Map> collectIntervalsAndShardSpecs( @@ -816,6 +833,29 @@ private Map> collectIntervalsAndShardSp return hllCollectors; } + private IndexTaskSegmentAllocator createSegmentAllocator( + TaskToolbox toolbox, + DataSchema dataSchema, + Map> allocateSpec + ) throws IOException + { + if (ingestionSchema.ioConfig.isAppendToExisting() || !isChangeSegmentGranularity()) { + if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { + return new CachingRemoteSegmentAllocator(toolbox, getId(), allocateSpec); + } else { + return new RemoteSegmentAllocator(toolbox, getId(), dataSchema, isOverwriteMode(), isChangeSegmentGranularity(), getAllOverwritingSegmentMeta()); + } + } else { + // We use the timeChunk lock and don't have to ask the overlord to create segmentIds. + // Instead, a local allocator is used. + if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { + return new CachingLocalSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); + } else { + return new LocalSegmentAllocator(toolbox, getId(), getDataSource(), dataSchema.getGranularitySpec()); + } + } + } + /** * This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}. * If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs @@ -837,8 +877,7 @@ private Map> collectIntervalsAndShardSp private TaskStatus generateAndPublishSegments( final TaskToolbox toolbox, final DataSchema dataSchema, - final ShardSpecs shardSpecs, - final Map versions, + final Map> allocateSpec, final FirehoseFactory firehoseFactory, final File firehoseTempDir, @Nullable final Integer maxRowsPerSegment, @@ -860,66 +899,15 @@ private TaskStatus generateAndPublishSegments( } final IndexIOConfig ioConfig = ingestionSchema.getIOConfig(); - final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig; + final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final long pushTimeout = tuningConfig.getPushTimeout(); final boolean isGuaranteedRollup = isGuaranteedRollup(ioConfig, tuningConfig); - final SegmentAllocator segmentAllocator; - if (isGuaranteedRollup) { - // Overwrite mode, guaranteed rollup: segments are all known in advance and there is one per sequenceName. - final Map lookup = new HashMap<>(); - - for (Map.Entry> entry : shardSpecs.getMap().entrySet()) { - for (ShardSpec shardSpec : entry.getValue()) { - final String version = findVersion(versions, entry.getKey()); - lookup.put( - Appenderators.getSequenceName(entry.getKey(), version, shardSpec), - new SegmentIdWithShardSpec(getDataSource(), entry.getKey(), version, shardSpec) - ); - } - } - - segmentAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> lookup.get(sequenceName); - } else if (ioConfig.isAppendToExisting()) { - // Append mode: Allocate segments as needed using Overlord APIs. - segmentAllocator = new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ); - } else { - // Overwrite mode, non-guaranteed rollup: We can make up our own segment ids but we don't know them in advance. - final Map counters = new HashMap<>(); - - segmentAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> { - final DateTime timestamp = row.getTimestamp(); - Optional maybeInterval = granularitySpec.bucketInterval(timestamp); - if (!maybeInterval.isPresent()) { - throw new ISE("Could not find interval for timestamp [%s]", timestamp); - } - - final Interval interval = maybeInterval.get(); - if (!shardSpecs.getMap().containsKey(interval)) { - throw new ISE("Could not find shardSpec for interval[%s]", interval); - } - - final int partitionNum = counters.computeIfAbsent(interval, x -> new AtomicInteger()).getAndIncrement(); - return new SegmentIdWithShardSpec( - getDataSource(), - interval, - findVersion(versions, interval), - new NumberedShardSpec(partitionNum, 0) - ); - }; - } + final IndexTaskSegmentAllocator segmentAllocator = createSegmentAllocator( + toolbox, + dataSchema, + allocateSpec + ); final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); @@ -936,7 +924,7 @@ private TaskStatus generateAndPublishSegments( final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { - driver.startJob(); + driver.startJob(null); while (firehose.hasMore()) { try { @@ -966,8 +954,7 @@ private TaskStatus generateAndPublishSegments( if (isGuaranteedRollup) { // Sequence name is based solely on the shardSpec, and there will only be one segment per sequence. final Interval interval = optInterval.get(); - final ShardSpec shardSpec = shardSpecs.getShardSpec(interval, inputRow); - sequenceName = Appenderators.getSequenceName(interval, findVersion(versions, interval), shardSpec); + sequenceName = segmentAllocator.getSequenceName(interval, inputRow); } else { // Segments are created as needed, using a single sequence name. They may be allocated from the overlord // (in append mode) or may be created on our own authority (in overwrite mode). @@ -1002,6 +989,7 @@ private TaskStatus generateAndPublishSegments( final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); log.info("Pushed segments[%s]", pushed.getSegments()); + // Probably we can publish atomicUpdateGroup along with segments. final SegmentsAndMetadata published = awaitPublish( driver.publishAll(publisher), pushTimeout @@ -1176,6 +1164,7 @@ ShardSpec getShardSpec(Interval interval, InputRow row) if (shardSpecs == null || shardSpecs.isEmpty()) { throw new ISE("Failed to get shardSpec for interval[%s]", interval); } + // TODO: cache shardSpecLookup return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(row.getTimestampFromEpoch(), row); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java new file mode 100644 index 000000000000..3cd87c627ad8 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.indexing.common.task.IndexTask.SequenceNameGenerator; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; + +// TODO: rename??? +public interface IndexTaskSegmentAllocator extends SegmentAllocator, SequenceNameGenerator +{ +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java index 44c90bc8a47b..b2da38364d11 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java @@ -22,19 +22,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentNukeAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionPreconditions; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; +import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.stream.Collectors; /** */ @@ -65,36 +74,54 @@ public String getType() } @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception + public boolean requireLockInputSegments() { - // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient())); + return true; + } - if (!myLock.getDataSource().equals(getDataSource())) { - throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + throws IOException + { + final List allSegments = new ArrayList<>(); + for (Interval interval : intervals) { + allSegments.addAll(taskActionClient.submit(new SegmentListUnusedAction(getDataSource(), interval))); } + return allSegments; + } - if (!myLock.getInterval().equals(getInterval())) { - throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval()); - } + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final NavigableMap> taskLockMap = getTaskLockMap(toolbox.getTaskActionClient()); // List unused segments final List unusedSegments = toolbox .getTaskActionClient() - .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); - - // Verify none of these segments have versions > lock version - for (final DataSegment unusedSegment : unusedSegments) { - if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { - throw new ISE( - "WTF?! Unused segment[%s] has version[%s] > task version[%s]", - unusedSegment.getId(), - unusedSegment.getVersion(), - myLock.getVersion() - ); - } - - log.info("OK to kill segment: %s", unusedSegment.getId()); + .submit(new SegmentListUnusedAction(getDataSource(), getInterval())); + + // log.info("segments to kill: %s", unusedSegments); + // log.info("taskLockMap: %s", taskLockMap); + + if (!TaskActionPreconditions.isLockCoversSegments(taskLockMap, unusedSegments)) { + throw new ISE( + "Locks[%s] for task[%s] can't cover segments[%s]", + taskLockMap.values().stream().flatMap(List::stream).collect(Collectors.toList()), + getId(), + unusedSegments + ); } // Kill segments @@ -105,4 +132,13 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } + + private NavigableMap> getTaskLockMap(TaskActionClient client) throws IOException + { + final NavigableMap> taskLockMap = new TreeMap<>(); + getTaskLocks(client).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/task/LocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java new file mode 100644 index 000000000000..cbaf7026e11d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java @@ -0,0 +1,99 @@ +/* + * 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.Optional; +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.java.util.common.ISE; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +public class LocalSegmentAllocator implements IndexTaskSegmentAllocator +{ + private final String taskId; + + private final SegmentAllocator internalAllocator; + + public LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec) + throws IOException + { + this.taskId = taskId; + final Map counters = new HashMap<>(); + + final Map intervalToVersion = toolbox.getTaskActionClient() + .submit(new LockListAction()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + + internalAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> { + final DateTime timestamp = row.getTimestamp(); + Optional maybeInterval = granularitySpec.bucketInterval(timestamp); + if (!maybeInterval.isPresent()) { + throw new ISE("Could not find interval for timestamp [%s]", timestamp); + } + + final Interval interval = maybeInterval.get(); + final String version = intervalToVersion.entrySet().stream() + .filter(entry -> entry.getKey().contains(interval)) + .map(Entry::getValue) + .findFirst() + .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); + + final int partitionNum = counters.computeIfAbsent(interval, x -> new AtomicInteger()).getAndIncrement(); + return new SegmentIdWithShardSpec( + dataSource, + interval, + version, + new NumberedShardSpec(partitionNum, 0) + ); + }; + } + + @Override + public String getSequenceName(Interval interval, InputRow inputRow) + { + return taskId; + } + + @Override + public SegmentIdWithShardSpec allocate( + InputRow row, + String sequenceName, + String previousSegmentId, + boolean skipSegmentLineageCheck + ) throws IOException + { + return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java index 6bdaef816c1a..432aa50d4ae4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java @@ -28,11 +28,15 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -110,6 +114,31 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @JsonProperty("target") public Map getTargetLoadSpec() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java index cc9ed629be31..02126093e2c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java @@ -32,8 +32,14 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; 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.timeline.DataSegment; +import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.UUID; @@ -151,6 +157,31 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception } } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public int getPriority() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 2786805b9f4c..c184bede2cdf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -38,12 +38,13 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.LockAcquireAction; import org.apache.druid.indexing.common.actions.LockReleaseAction; 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.java.util.common.DateTimes; 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.CloseQuietly; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.FinalizeResultsQueryRunner; @@ -73,8 +74,11 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; +import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Timer; import java.util.TimerTask; @@ -238,9 +242,10 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception public void announceSegment(final DataSegment segment) throws IOException { // Side effect: Calling announceSegment causes a lock to be acquired + // TODO: get lock before allocating segment Preconditions.checkNotNull( toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) ), "Cannot acquire a lock for interval[%s]", segment.getInterval() @@ -263,10 +268,11 @@ public void unannounceSegment(final DataSegment segment) throws IOException public void announceSegments(Iterable segments) throws IOException { // Side effect: Calling announceSegments causes locks to be acquired + // TODO: get lock before allocating segment for (DataSegment segment : segments) { Preconditions.checkNotNull( toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) + new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) ), "Cannot acquire a lock for interval[%s]", segment.getInterval() @@ -302,7 +308,8 @@ public String getVersion(final Interval interval) { try { // Side effect: Calling getVersion causes a lock to be acquired - final LockAcquireAction action = new LockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); + // TODO: get lock to get the version? + final TimeChunkLockAcquireAction action = new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); final TaskLock lock = Preconditions.checkNotNull( toolbox.getTaskActionClient().submit(action), "Cannot acquire a lock for interval[%s]", @@ -486,6 +493,31 @@ public void run() return TaskStatus.success(getId()); } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public boolean canRestore() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java new file mode 100644 index 000000000000..0ec1ada6cd43 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.task.AbstractTask.OverwritingSegmentMeta; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; +import org.apache.druid.timeline.partition.NumberedShardSpecFactory; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.Map; + +public class RemoteSegmentAllocator implements IndexTaskSegmentAllocator +{ + private final String taskId; + private final ActionBasedSegmentAllocator internalAllocator; + + public RemoteSegmentAllocator( + TaskToolbox toolbox, + String taskId, + DataSchema dataSchema, + boolean isOverwriteMode, + boolean isChangeSegmentGranularity, + Map overwritingSegmentMetaMap + ) + { + this.taskId = taskId; + this.internalAllocator = new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + dataSchema, + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> { + final GranularitySpec granularitySpec = schema.getGranularitySpec(); + final Interval interval = granularitySpec + .bucketInterval(row.getTimestamp()) + .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); + final ShardSpecFactory shardSpecFactory; + if (isOverwriteMode && !isChangeSegmentGranularity) { + final OverwritingSegmentMeta overwritingSegmentMeta = overwritingSegmentMetaMap.get(interval); + if (overwritingSegmentMeta == null) { + throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); + } + shardSpecFactory = new NumberedOverwritingShardSpecFactory( + overwritingSegmentMeta.getStartRootPartitionId(), + overwritingSegmentMeta.getEndRootPartitionId(), + overwritingSegmentMeta.getMinorVersionForNewSegments() + ); + } else { + shardSpecFactory = NumberedShardSpecFactory.instance(); + } + return new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck, + shardSpecFactory + ); + } + ); + } + + @Override + public SegmentIdWithShardSpec allocate( + InputRow row, + String sequenceName, + String previousSegmentId, + boolean skipSegmentLineageCheck + ) throws IOException + { + return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck); + } + + @Override + public String getSequenceName(Interval interval, InputRow inputRow) + { + return taskId; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java index f3e98ae40289..d77a685a29ec 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java @@ -27,12 +27,16 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -119,4 +123,29 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } + + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index deebf3b38af5..c70b37df6ab5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -19,20 +19,12 @@ package org.apache.druid.indexing.common.task; -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.actions.LockTryAcquireAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.guava.Comparators; import org.joda.time.Interval; -import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.TimeUnit; @@ -48,20 +40,6 @@ public class Tasks public static final String PRIORITY_KEY = "priority"; public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout"; - public static Map tryAcquireExclusiveLocks(TaskActionClient client, SortedSet intervals) - throws IOException - { - final Map lockMap = new HashMap<>(); - for (Interval interval : computeCompactIntervals(intervals)) { - final TaskLock lock = Preconditions.checkNotNull( - client.submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)), - "Cannot acquire a lock for interval[%s]", interval - ); - lockMap.put(interval, lock); - } - return lockMap; - } - public static SortedSet computeCompactIntervals(SortedSet intervals) { final SortedSet compactIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 435de05892ff..7b9189995639 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import org.apache.commons.io.FileUtils; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.Firehose; @@ -32,10 +32,9 @@ 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.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.LockTryAcquireAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractTask; @@ -47,12 +46,15 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.RealtimeMetricsMonitor; @@ -62,8 +64,12 @@ 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.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; +import org.apache.druid.timeline.partition.NumberedShardSpecFactory; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -152,8 +158,7 @@ public boolean isReady(TaskActionClient taskActionClient) private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) { try { - tryAcquireExclusiveSurrogateLocks(actionClient, intervals); - return true; + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals)); } catch (Exception e) { log.error(e, "Failed to acquire locks for intervals[%s]", intervals); @@ -206,52 +211,125 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } - private void tryAcquireExclusiveSurrogateLocks( - TaskActionClient client, - SortedSet intervals - ) + @Override + public boolean requireLockInputSegments() + { + return !ingestionSchema.getIOConfig().isAppendToExisting(); + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) throws IOException { - for (Interval interval : Tasks.computeCompactIntervals(intervals)) { - Preconditions.checkNotNull( - client.submit( - new SurrogateAction<>(supervisorTaskId, new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)) - ), - "Cannot acquire a lock for interval[%s]", interval - ); - } + return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); } - private SegmentAllocator createSegmentAllocator( + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + } + + @VisibleForTesting + SegmentAllocator createSegmentAllocator( TaskToolbox toolbox, - ParallelIndexTaskClient taskClient, - ParallelIndexIngestionSpec ingestionSchema + ParallelIndexTaskClient taskClient ) { - final DataSchema dataSchema = ingestionSchema.getDataSchema(); - final ParallelIndexIOConfig ioConfig = ingestionSchema.getIOConfig(); - if (ioConfig.isAppendToExisting()) { - return new ActionBasedSegmentAllocator( - toolbox.getTaskActionClient(), - dataSchema, - (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SurrogateAction<>( - supervisorTaskId, - new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck - ) - ) - ); - } else { - return (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> taskClient.allocateSegment( - supervisorTaskId, - row.getTimestamp() - ); + return new WrappingSegmentAllocator(toolbox, taskClient); + } + + private class WrappingSegmentAllocator implements SegmentAllocator + { + private final TaskToolbox toolbox; + private final ParallelIndexTaskClient taskClient; + + /** + * This internalAllocator is initialized lazily to make sure that {@link #isChangeSegmentGranularity()} is called + * after the lock is properly acquired. Note that locks can be acquired after the task is started if the interval is + * not specified in {@link GranularitySpec}. + */ + private SegmentAllocator internalAllocator; + + private WrappingSegmentAllocator( + TaskToolbox toolbox, + ParallelIndexTaskClient taskClient + ) + { + this.toolbox = toolbox; + this.taskClient = taskClient; + } + + @Override + public SegmentIdWithShardSpec allocate( + InputRow row, + String sequenceName, + String previousSegmentId, + boolean skipSegmentLineageCheck + ) throws IOException + { + if (internalAllocator == null) { + internalAllocator = createSegmentAllocator(); + } + return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck); + } + + private SegmentAllocator createSegmentAllocator() + { + // TODO: what if intervals are missing? + if (ingestionSchema.getIOConfig().isAppendToExisting() || !isChangeSegmentGranularity()) { + return new ActionBasedSegmentAllocator( + toolbox.getTaskActionClient(), + ingestionSchema.getDataSchema(), + (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> { + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + final Interval interval = granularitySpec + .bucketInterval(row.getTimestamp()) + .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); + final ShardSpecFactory shardSpecFactory; + if (isOverwriteMode() && !isChangeSegmentGranularity()) { + final OverwritingSegmentMeta overwritingSegmentMeta = getOverwritingSegmentMeta(interval); + if (overwritingSegmentMeta == null) { + throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); + } + shardSpecFactory = new NumberedOverwritingShardSpecFactory( + overwritingSegmentMeta.getStartRootPartitionId(), + overwritingSegmentMeta.getEndRootPartitionId(), + overwritingSegmentMeta.getMinorVersionForNewSegments() + ); + } else { + shardSpecFactory = NumberedShardSpecFactory.instance(); + } + + return new SurrogateAction<>( + supervisorTaskId, + new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck, + shardSpecFactory + ) + ); + } + ); + } else { + return (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> taskClient.allocateSegment( + supervisorTaskId, + row.getTimestamp() + ); + } } } @@ -301,14 +379,14 @@ private List generateAndPushSegments( @Nullable final Long maxTotalRows = IndexTask.getValidMaxTotalRows(tuningConfig); final long pushTimeout = tuningConfig.getPushTimeout(); final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); - final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient, ingestionSchema); + final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); try ( final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { - driver.startJob(); + driver.startJob(null); final List pushedSegments = new ArrayList<>(); @@ -335,6 +413,12 @@ private List generateAndPushSegments( fireDepartmentMetrics.incrementThrownAway(); continue; } + } else { + final Granularity segmentGranularity = findSegmentGranularity(granularitySpec); + final Interval timeChunk = segmentGranularity.bucket(inputRow.getTimestamp()); + if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk))) { + throw new ISE("Failed to get locks for interval[%s]", timeChunk); + } } // Segments are created as needed, using a single sequence name. They may be allocated from the overlord @@ -377,6 +461,15 @@ private List generateAndPushSegments( } } + private static Granularity findSegmentGranularity(GranularitySpec granularitySpec) + { + if (granularitySpec instanceof UniformGranularitySpec) { + return granularitySpec.getSegmentGranularity(); + } else { + return Granularities.ALL; + } + } + private static Appenderator newAppenderator( FireDepartmentMetrics metrics, TaskToolbox toolbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 2a29726965c4..6285cea27cbf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -33,10 +33,9 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.Counters; import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; -import org.apache.druid.indexing.common.actions.LockTryAcquireAction; +import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -50,6 +49,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -59,6 +59,7 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlers; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -76,6 +77,7 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -190,6 +192,7 @@ AuthorizerMapper getAuthorizerMapper() @VisibleForTesting ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) { + this.toolbox = toolbox; if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { throw new UnsupportedOperationException("Perfect roll-up is not supported yet"); } else { @@ -218,22 +221,34 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception .getGranularitySpec() .bucketIntervals(); - return !intervals.isPresent() || isReady(taskActionClient, intervals.get()); + return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get())); } - static boolean isReady(TaskActionClient actionClient, SortedSet intervals) throws IOException + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + throws IOException { - final List locks = getTaskLocks(actionClient); - if (locks.isEmpty()) { - try { - Tasks.tryAcquireExclusiveLocks(actionClient, intervals); - } - catch (Exception e) { - log.error(e, "Failed to acquire locks for intervals[%s]", intervals); - return false; - } - } - return true; + return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); + } + + @Override + public boolean requireLockInputSegments() + { + return !ingestionSchema.getIOConfig().isAppendToExisting(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); } @Override @@ -247,8 +262,6 @@ public void stopGracefully(TaskConfig taskConfig) @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - setToolbox(toolbox); - log.info( "Found chat handler of class[%s]", Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName() @@ -387,79 +400,43 @@ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException { final String dataSource = getDataSource(); final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec(); - final Optional> bucketIntervals = granularitySpec.bucketIntervals(); - + final SortedSet bucketIntervals = Preconditions.checkNotNull( + granularitySpec.bucketIntervals().orNull(), + "bucketIntervals" + ); // List locks whenever allocating a new segment because locks might be revoked and no longer valid. - final List locks = toolbox + final Map versions = toolbox .getTaskActionClient() - .submit(new LockListAction()); - final TaskLock revokedLock = locks.stream().filter(TaskLock::isRevoked).findAny().orElse(null); - if (revokedLock != null) { - throw new ISE("Lock revoked: [%s]", revokedLock); - } - final Map versions = locks + .submit(new LockListAction()) .stream() .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - Interval interval; - String version; - boolean justLockedInterval = false; - if (bucketIntervals.isPresent()) { - // If the granularity spec has explicit intervals, we just need to find the interval (of the segment - // granularity); we already tried to lock it at task startup. - final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); - if (!maybeInterval.isPresent()) { - throw new IAE("Could not find interval for timestamp [%s]", timestamp); - } - - interval = maybeInterval.get(); - if (!bucketIntervals.get().contains(interval)) { - throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); - } + final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); + if (!maybeInterval.isPresent()) { + throw new IAE("Could not find interval for timestamp [%s]", timestamp); + } - version = findVersion(versions, interval); - if (version == null) { - throw new ISE("Cannot find a version for interval[%s]", interval); - } - } else { - // We don't have explicit intervals. We can use the segment granularity to figure out what - // interval we need, but we might not have already locked it. - interval = granularitySpec.getSegmentGranularity().bucket(timestamp); - version = findVersion(versions, interval); - if (version == null) { - // We don't have a lock for this interval, so we should lock it now. - final TaskLock lock = Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit(new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)), - "Cannot acquire a lock for interval[%s]", interval - ); - version = lock.getVersion(); - justLockedInterval = true; - } + final Interval interval = maybeInterval.get(); + if (!bucketIntervals.contains(interval)) { + throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); } final int partitionNum = Counters.getAndIncrementInt(partitionNumCountersPerInterval, interval); - if (justLockedInterval && partitionNum != 0) { - throw new ISE( - "Expected partitionNum to be 0 for interval [%s] right after locking, but got [%s]", - interval, partitionNum - ); - } return new SegmentIdWithShardSpec( dataSource, interval, - version, + findVersion(versions, interval), new NumberedShardSpec(partitionNum, 0) ); } - @Nullable private static String findVersion(Map versions, Interval interval) { return versions.entrySet().stream() .filter(entry -> entry.getKey().contains(interval)) .map(Entry::getValue) .findFirst() - .orElse(null); + .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java index 9b06a4febc8c..1115b7398f8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java @@ -29,7 +29,7 @@ * middle of the action. * * Implementations must not change the lock state by calling {@link TaskLockbox#lock(TaskLockType, Task, Interval)}, - * {@link TaskLockbox#lock(TaskLockType, Task, Interval, long)}, {@link TaskLockbox#tryLock(TaskLockType, Task, Interval)}, + * {@link TaskLockbox#lock(TaskLockType, Task, Interval, long)}, {@link TaskLockbox#tryTimeChunkLock(TaskLockType, Task, Interval)}, * or {@link TaskLockbox#unlock(Task, Interval)}. * * Also, implementations should be finished as soon as possible because all methods in {@link TaskLockbox} are blocked diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequest.java new file mode 100644 index 000000000000..987139f5be5b --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequest.java @@ -0,0 +1,46 @@ +/* + * 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.overlord; + +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.joda.time.Interval; + +public interface LockRequest +{ + LockGranularity getGranularity(); + + TaskLockType getType(); + + String getGroupId(); + + String getDataSource(); + + Interval getInterval(); + + String getVersion(); + + int getPriority(); + + boolean isRevoked(); + + TaskLock toLock(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java new file mode 100644 index 000000000000..1edb3f7dc57e --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java @@ -0,0 +1,183 @@ +/* + * 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.overlord; + +import com.google.common.annotations.VisibleForTesting; +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.task.Task; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.joda.time.Interval; + +import javax.annotation.Nullable; + +public class LockRequestForNewSegment implements LockRequest +{ + private final TaskLockType lockType; + private final String groupId; + private final String dataSource; + private final Interval interval; + private final ShardSpecFactory shardSpecFactory; + private final int priority; + private final String sequenceName; + @Nullable + private final String previsousSegmentId; + private final boolean skipSegmentLineageCheck; + + public LockRequestForNewSegment( + TaskLockType lockType, + String groupId, + String dataSource, + Interval interval, + ShardSpecFactory shardSpecFactory, + int priority, + String sequenceName, + @Nullable String previsousSegmentId, + boolean skipSegmentLineageCheck + ) + { + this.lockType = lockType; + this.groupId = groupId; + this.dataSource = dataSource; + this.interval = interval; + this.shardSpecFactory = shardSpecFactory; + this.priority = priority; + this.sequenceName = sequenceName; + this.previsousSegmentId = previsousSegmentId; + this.skipSegmentLineageCheck = skipSegmentLineageCheck; + } + + @VisibleForTesting + public LockRequestForNewSegment( + TaskLockType lockType, + Task task, + Interval interval, + ShardSpecFactory shardSpecFactory, + String sequenceName, + @Nullable String previsousSegmentId, + boolean skipSegmentLineageCheck + ) + { + this( + lockType, + task.getGroupId(), + task.getDataSource(), + interval, + shardSpecFactory, + task.getPriority(), + sequenceName, + previsousSegmentId, + skipSegmentLineageCheck + ); + } + + @Override + public LockGranularity getGranularity() + { + return LockGranularity.SEGMENT; + } + + @Override + public TaskLockType getType() + { + return lockType; + } + + @Override + public String getGroupId() + { + return groupId; + } + + @Override + public String getDataSource() + { + return dataSource; + } + + @Override + public Interval getInterval() + { + return interval; + } + + @Override + public int getPriority() + { + return priority; + } + + public ShardSpecFactory getShardSpecFactory() + { + return shardSpecFactory; + } + + @Override + public String getVersion() + { + return DateTimes.nowUtc().toString(); + } + + @Override + public boolean isRevoked() + { + return false; + } + + @Override + public TaskLock toLock() + { + throw new UnsupportedOperationException(); + } + + public String getSequenceName() + { + return sequenceName; + } + + @Nullable + public String getPrevisousSegmentId() + { + return previsousSegmentId; + } + + public boolean isSkipSegmentLineageCheck() + { + return skipSegmentLineageCheck; + } + + @Override + public String toString() + { + return "LockRequestForNewSegment{" + + "lockType=" + lockType + + ", groupId='" + groupId + '\'' + + ", dataSource='" + dataSource + '\'' + + ", interval=" + interval + + ", shardSpecFactory=" + shardSpecFactory + + ", priority=" + priority + + ", sequenceName='" + sequenceName + '\'' + + ", previsousSegmentId='" + previsousSegmentId + '\'' + + ", skipSegmentLineageCheck=" + skipSegmentLineageCheck + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java index 6372642b8fb6..4344fe324e98 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java @@ -22,54 +22,63 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.indexing.common.task.Task; -import org.joda.time.Interval; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import javax.annotation.Nullable; /** - * This class represents the result of {@link TaskLockbox#tryLock(TaskLockType, Task, Interval)}. If the lock + * This class represents the result of {@link TaskLockbox#tryLock}. If the lock * acquisition fails, the callers can tell that it was failed because it was preempted by other locks of higher * priorities or not by checking the {@link #revoked} flag. * * The {@link #revoked} flag means that consecutive lock acquisitions for the same dataSource and interval are * returning different locks because another lock of a higher priority preempted your lock at some point. In this case, * the lock acquisition must fail. - * - * @see TaskLockbox#tryLock(TaskLockType, Task, Interval) */ public class LockResult { + @Nullable private final TaskLock taskLock; private final boolean revoked; + @Nullable + private final SegmentIdWithShardSpec newSegmentId; - public static LockResult ok(TaskLock taskLock) + public static LockResult ok(TaskLock taskLock, SegmentIdWithShardSpec newSegmentId) { - return new LockResult(taskLock, false); + return new LockResult(taskLock, newSegmentId, false); } public static LockResult fail(boolean revoked) { - return new LockResult(null, revoked); + return new LockResult(null, null, revoked); } @JsonCreator public LockResult( @JsonProperty("taskLock") @Nullable TaskLock taskLock, + @JsonProperty("newSegmentId") @Nullable SegmentIdWithShardSpec newSegmentId, @JsonProperty("revoked") boolean revoked ) { this.taskLock = taskLock; + this.newSegmentId = newSegmentId; this.revoked = revoked; } @JsonProperty("taskLock") + @Nullable public TaskLock getTaskLock() { return taskLock; } + @JsonProperty("newSegmentId") + @Nullable + public SegmentIdWithShardSpec getNewSegmentId() + { + return newSegmentId; + } + @JsonProperty("revoked") public boolean isRevoked() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SpecificSegmentLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SpecificSegmentLockRequest.java new file mode 100644 index 000000000000..8b20f6c52fb3 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/SpecificSegmentLockRequest.java @@ -0,0 +1,172 @@ +/* + * 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.overlord; + +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.TaskLockType; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.joda.time.Interval; + +public class SpecificSegmentLockRequest implements LockRequest +{ + private final TaskLockType lockType; + private final String groupId; + private final String dataSource; + private final Interval interval; + private final int partitionId; + private final String version; + private final int priority; + private final boolean revoked; + + public SpecificSegmentLockRequest( + TaskLockType lockType, + String groupId, + String dataSource, + Interval interval, + String version, + int partitionId, + int priority, + boolean revoked + ) + { + this.lockType = lockType; + this.groupId = groupId; + this.dataSource = dataSource; + this.interval = interval; + this.version = version; + this.partitionId = partitionId; + this.priority = priority; + this.revoked = revoked; + } + + public SpecificSegmentLockRequest( + TaskLockType lockType, + Task task, + Interval interval, + String version, + int partitionId + ) + { + this(lockType, task.getGroupId(), task.getDataSource(), interval, version, partitionId, task.getPriority(), false); + } + + public SpecificSegmentLockRequest( + LockRequestForNewSegment request, + SegmentIdWithShardSpec newId + ) + { + this( + request.getType(), + request.getGroupId(), + newId.getDataSource(), + newId.getInterval(), + newId.getVersion(), + newId.getShardSpec().getPartitionNum(), + request.getPriority(), + request.isRevoked() + ); + } + + @Override + public LockGranularity getGranularity() + { + return LockGranularity.SEGMENT; + } + + @Override + public TaskLockType getType() + { + return lockType; + } + + @Override + public String getGroupId() + { + return groupId; + } + + @Override + public String getDataSource() + { + return dataSource; + } + + @Override + public Interval getInterval() + { + return interval; + } + + @Override + public String getVersion() + { + return version; + } + + @Override + public int getPriority() + { + return priority; + } + + @Override + public boolean isRevoked() + { + return revoked; + } + + public int getPartitionId() + { + return partitionId; + } + + @Override + public TaskLock toLock() + { + return new SegmentLock( + lockType, + groupId, + dataSource, + interval, + version, + partitionId, + priority, + revoked + ); + } + + @Override + public String toString() + { + return "SpecificSegmentLockRequest{" + + "lockType=" + lockType + + ", groupId='" + groupId + '\'' + + ", dataSource='" + dataSource + '\'' + + ", interval=" + interval + + ", partitionId=" + partitionId + + ", version='" + version + '\'' + + ", priority=" + priority + + ", revoked=" + revoked + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 787852ddfbd5..0aa9247b7b9d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -29,15 +29,17 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.inject.Inject; +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.TaskLockType; +import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.DateTimes; 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.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -65,6 +67,10 @@ * Remembers which activeTasks have locked which intervals. Tasks are permitted to lock an interval if no other task * outside their group has locked an overlapping interval for the same datasource. When a task locks an interval, * it is assigned a version string that it can use to publish segments. + * + * // TODO: how to use this? + * // with segemnt lock + * // note it also does segment allocation for segment lock */ public class TaskLockbox { @@ -77,6 +83,7 @@ public class TaskLockbox private final Map>>> running = new HashMap<>(); private final TaskStorage taskStorage; + private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final ReentrantLock giant = new ReentrantLock(true); private final Condition lockReleaseCondition = giant.newCondition(); @@ -88,10 +95,12 @@ public class TaskLockbox @Inject public TaskLockbox( - TaskStorage taskStorage + TaskStorage taskStorage, + IndexerMetadataStorageCoordinator metadataStorageCoordinator ) { this.taskStorage = taskStorage; + this.metadataStorageCoordinator = metadataStorageCoordinator; } /** @@ -130,8 +139,8 @@ public int compare(Pair left, Pair right) // Bookkeeping for a log message at the end int taskLockCount = 0; for (final Pair taskAndLock : byVersionOrdering.sortedCopy(storedLocks)) { - final Task task = taskAndLock.lhs; - final TaskLock savedTaskLock = taskAndLock.rhs; + final Task task = Preconditions.checkNotNull(taskAndLock.lhs, "task"); + final TaskLock savedTaskLock = Preconditions.checkNotNull(taskAndLock.rhs, "savedTaskLock"); if (savedTaskLock.getInterval().toDurationMillis() <= 0) { // "Impossible", but you never know what crazy stuff can be restored from storage. log.warn("WTF?! Got lock[%s] with empty interval for task: %s", savedTaskLock, task.getId()); @@ -141,10 +150,13 @@ public int compare(Pair left, Pair right) // Create a new taskLock if it doesn't have a proper priority, // so that every taskLock in memory has the priority. final TaskLock savedTaskLockWithPriority = savedTaskLock.getPriority() == null - ? TaskLock.withPriority(savedTaskLock, task.getPriority()) + ? savedTaskLock.withPriority(task.getPriority()) : savedTaskLock; - final TaskLockPosse taskLockPosse = createOrFindLockPosse(task, savedTaskLockWithPriority); + final TaskLockPosse taskLockPosse = verifyAndCreateOrFindLockPosse( + task, + savedTaskLockWithPriority + ); if (taskLockPosse != null) { taskLockPosse.addTask(task); @@ -188,9 +200,79 @@ public int compare(Pair left, Pair right) } } + /** + * This method is called only in {@link #syncFromStorage()} and verifies the given task and the taskLock have the same + * groupId, dataSource, and priority. + */ + private TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) + { + giant.lock(); + + try { + Preconditions.checkArgument( + task.getGroupId().equals(taskLock.getGroupId()), + "lock groupId[%s] is different from task groupId[%s]", + taskLock.getGroupId(), + task.getGroupId() + ); + Preconditions.checkArgument( + task.getDataSource().equals(taskLock.getDataSource()), + "lock dataSource[%s] is different from task dataSource[%s]", + taskLock.getDataSource(), + task.getDataSource() + ); + final int taskPriority = task.getPriority(); + final int lockPriority = taskLock.getNonNullPriority(); + + Preconditions.checkArgument( + lockPriority == taskPriority, + "lock priority[%s] is different from task priority[%s]", + lockPriority, + taskPriority + ); + + final LockRequest request; + switch (taskLock.getGranularity()) { + case SEGMENT: + final SegmentLock segmentLock = (SegmentLock) taskLock; + request = new SpecificSegmentLockRequest( + segmentLock.getLockType(), + segmentLock.getGroupId(), + segmentLock.getDataSource(), + segmentLock.getInterval(), + segmentLock.getVersion(), + segmentLock.getPartitionId(), + taskPriority, + segmentLock.isRevoked() + ); + break; + case TIME_CHUNK: + final TimeChunkLock timeChunkLock = (TimeChunkLock) taskLock; + request = new TimeChunkLockRequest( + timeChunkLock.getLockType(), + timeChunkLock.getGroupId(), + timeChunkLock.getDataSource(), + timeChunkLock.getInterval(), + timeChunkLock.getVersion(), + taskPriority, + timeChunkLock.isRevoked() + ); + break; + default: + throw new ISE("Unknown lockGranularity[%s]", taskLock.getGranularity()); + } + + return createOrFindLockPosse(task, request); + } + finally { + giant.unlock(); + } + } + /** * Acquires a lock on behalf of a task. Blocks until the lock is acquired. * + * @param granularity lock granularity * @param lockType lock type * @param task task to acquire lock for * @param interval interval to lock @@ -200,16 +282,12 @@ public int compare(Pair left, Pair right) * * @throws InterruptedException if the current thread is interrupted */ - public LockResult lock( - final TaskLockType lockType, - final Task task, - final Interval interval - ) throws InterruptedException + public LockResult lock(final Task task, final LockRequest request) throws InterruptedException { giant.lockInterruptibly(); try { LockResult lockResult; - while (!(lockResult = tryLock(lockType, task, interval)).isOk()) { + while (!(lockResult = tryLock(task, request)).isOk()) { if (lockResult.isRevoked()) { return lockResult; } @@ -225,6 +303,7 @@ public LockResult lock( /** * Acquires a lock on behalf of a task, waiting up to the specified wait time if necessary. * + * @param granularity lock granularity * @param lockType lock type * @param task task to acquire a lock for * @param interval interval to lock @@ -235,18 +314,13 @@ public LockResult lock( * * @throws InterruptedException if the current thread is interrupted */ - public LockResult lock( - final TaskLockType lockType, - final Task task, - final Interval interval, - long timeoutMs - ) throws InterruptedException + public LockResult lock(final Task task, final LockRequest request, long timeoutMs) throws InterruptedException { long nanos = TimeUnit.MILLISECONDS.toNanos(timeoutMs); giant.lockInterruptibly(); try { LockResult lockResult; - while (!(lockResult = tryLock(lockType, task, interval)).isOk()) { + while (!(lockResult = tryLock(task, request)).isOk()) { if (nanos <= 0 || lockResult.isRevoked()) { return lockResult; } @@ -263,6 +337,7 @@ public LockResult lock( * Attempt to acquire a lock for a task, without removing it from the queue. Can safely be called multiple times on * the same task until the lock is preempted. * + * @param granularity lock granularity * @param lockType type of lock to be acquired * @param task task that wants a lock * @param interval interval to lock @@ -272,11 +347,7 @@ public LockResult lock( * * @throws IllegalStateException if the task is not a valid active task */ - public LockResult tryLock( - final TaskLockType lockType, - final Task task, - final Interval interval - ) + public LockResult tryLock(final Task task, final LockRequest request) { giant.lock(); @@ -284,18 +355,32 @@ public LockResult tryLock( if (!activeTasks.contains(task.getId())) { throw new ISE("Unable to grant lock to inactive Task [%s]", task.getId()); } - Preconditions.checkArgument(interval.toDurationMillis() > 0, "interval empty"); + Preconditions.checkArgument(request.getInterval().toDurationMillis() > 0, "interval empty"); + + final SegmentIdWithShardSpec newSegmentId; + final LockRequest convertedRequest; + if (request instanceof LockRequestForNewSegment) { + final LockRequestForNewSegment lockRequestForNewSegment = (LockRequestForNewSegment) request; + newSegmentId = allocateSegmentId(lockRequestForNewSegment); + if (newSegmentId == null) { + return LockResult.fail(false); + } + convertedRequest = new SpecificSegmentLockRequest(lockRequestForNewSegment, newSegmentId); + } else { + newSegmentId = null; + convertedRequest = request; + } - final TaskLockPosse posseToUse = createOrFindLockPosse(task, interval, lockType); + final TaskLockPosse posseToUse = createOrFindLockPosse(task, convertedRequest); if (posseToUse != null && !posseToUse.getTaskLock().isRevoked()) { // Add to existing TaskLockPosse, if necessary if (posseToUse.addTask(task)) { - log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId()); + log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock()); // Update task storage facility. If it fails, revoke the lock. try { taskStorage.addLock(task.getId(), posseToUse.getTaskLock()); - return LockResult.ok(posseToUse.getTaskLock()); + return LockResult.ok(posseToUse.getTaskLock(), newSegmentId); } catch (Exception e) { log.makeAlert("Failed to persist lock in storage") @@ -304,12 +389,18 @@ public LockResult tryLock( .addData("interval", posseToUse.getTaskLock().getInterval()) .addData("version", posseToUse.getTaskLock().getVersion()) .emit(); - unlock(task, interval); + unlock( + task, + convertedRequest.getInterval(), + posseToUse.getTaskLock().getGranularity() == LockGranularity.SEGMENT + ? ((SegmentLock) posseToUse.taskLock).getPartitionId() + : null + ); return LockResult.fail(false); } } else { log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId()); - return LockResult.ok(posseToUse.getTaskLock()); + return LockResult.ok(posseToUse.getTaskLock(), newSegmentId); } } else { final boolean lockRevoked = posseToUse != null && posseToUse.getTaskLock().isRevoked(); @@ -321,226 +412,81 @@ public LockResult tryLock( } } - /** - * See {@link #createOrFindLockPosse(Task, Interval, String, TaskLockType)} - */ - @Nullable - private TaskLockPosse createOrFindLockPosse( - final Task task, - final Interval interval, - final TaskLockType lockType - ) - { - giant.lock(); - - try { - return createOrFindLockPosse(task, interval, null, lockType); - } - finally { - giant.unlock(); - } - } - - /** - * Create a new {@link TaskLockPosse} or find an existing one for the given task and interval. Note that the returned - * {@link TaskLockPosse} can hold a revoked lock. - * - * @param task task acquiring a lock - * @param interval interval to be locked - * @param preferredVersion a preferred version string - * @param lockType type of lock to be acquired - * - * @return a lock posse or null if any posse is found and a new poss cannot be created - * - * @see #createNewTaskLockPosse - */ - @Nullable - private TaskLockPosse createOrFindLockPosse( - final Task task, - final Interval interval, - @Nullable final String preferredVersion, - final TaskLockType lockType - ) + private TaskLockPosse createOrFindLockPosse(Task task, LockRequest request) { - giant.lock(); + Preconditions.checkState(!(request instanceof LockRequestForNewSegment), "Can't handle LockRequestForNewSegment"); - try { - return createOrFindLockPosse( - lockType, - task.getId(), - task.getGroupId(), - task.getDataSource(), - interval, - preferredVersion, - task.getPriority(), - false - ); - } - finally { - giant.unlock(); - } - } - - @Nullable - private TaskLockPosse createOrFindLockPosse(Task task, TaskLock taskLock) - { giant.lock(); try { - Preconditions.checkArgument( - task.getGroupId().equals(taskLock.getGroupId()), - "lock groupId[%s] is different from task groupId[%s]", - taskLock.getGroupId(), - task.getGroupId() + final String taskId = task.getId(); + final List foundPosses = findLockPossesOverlapsInterval( + request.getDataSource(), + request.getInterval() ); - Preconditions.checkArgument( - task.getDataSource().equals(taskLock.getDataSource()), - "lock dataSource[%s] is different from task dataSource[%s]", - taskLock.getDataSource(), - task.getDataSource() - ); - final int taskPriority = task.getPriority(); - final int lockPriority = taskLock.getNonNullPriority(); - - Preconditions.checkArgument( - lockPriority == taskPriority, - "lock priority[%s] is different from task priority[%s]", - lockPriority, - taskPriority - ); - - return createOrFindLockPosse( - taskLock.getType(), - task.getId(), - taskLock.getGroupId(), - taskLock.getDataSource(), - taskLock.getInterval(), - taskLock.getVersion(), - taskPriority, - taskLock.isRevoked() - ); - } - finally { - giant.unlock(); - } - } - - @Nullable - private TaskLockPosse createOrFindLockPosse( - TaskLockType lockType, - String taskId, - String groupId, - String dataSource, - Interval interval, - @Nullable String preferredVersion, - int priority, - boolean revoked - ) - { - giant.lock(); - try { - final List foundPosses = findLockPossesOverlapsInterval(dataSource, interval); + final List conflictPosses = foundPosses + .stream() + .filter(taskLockPosse -> taskLockPosse.getTaskLock().conflict(request)) + .collect(Collectors.toList()); - if (foundPosses.size() > 0) { + if (conflictPosses.size() > 0) { // If we have some locks for dataSource and interval, check they can be reused. // If they can't be reused, check lock priority and revoke existing locks if possible. - final List filteredPosses = foundPosses + final List reusablePosses = foundPosses .stream() - .filter(posse -> matchGroupIdAndContainInterval(posse.taskLock, groupId, interval)) + .filter(posse -> posse.reusableFor(request)) .collect(Collectors.toList()); - if (filteredPosses.size() == 0) { + if (reusablePosses.size() == 0) { // case 1) this task doesn't have any lock, but others do - if (lockType.equals(TaskLockType.SHARED) && isAllSharedLocks(foundPosses)) { + if (request.getType().equals(TaskLockType.SHARED) && isAllSharedLocks(conflictPosses)) { // Any number of shared locks can be acquired for the same dataSource and interval. - return createNewTaskLockPosse( - lockType, - groupId, - dataSource, - interval, - preferredVersion, - priority, - revoked - ); + return createNewTaskLockPosse(request); } else { - if (isAllRevocable(foundPosses, priority)) { + if (isAllRevocable(conflictPosses, request.getPriority())) { // Revoke all existing locks - foundPosses.forEach(this::revokeLock); - - return createNewTaskLockPosse( - lockType, - groupId, - dataSource, - interval, - preferredVersion, - priority, - revoked - ); - } else { - final String messagePrefix; - if (preferredVersion == null) { - messagePrefix = StringUtils.format( - "Cannot create a new taskLockPosse for task[%s], interval[%s], priority[%d], revoked[%s]", - taskId, - interval, - priority, - revoked - ); - } else { - messagePrefix = StringUtils.format( - "Cannot create a new taskLockPosse for task[%s], interval[%s]," - + " preferredVersion[%s], priority[%d], revoked[%s]", - taskId, - interval, - preferredVersion, - priority, - revoked - ); - } + conflictPosses.forEach(this::revokeLock); + return createNewTaskLockPosse(request); + } else { log.info( - "%s because existing locks[%s] have same or higher priorities", - messagePrefix, - foundPosses + "Cannot create a new taskLockPosse for request[%s] because existing locks[%s] have same or higher priorities", + request, + conflictPosses ); return null; } } - } else if (filteredPosses.size() == 1) { + } else { // case 2) we found a lock posse for the given task - final TaskLockPosse foundPosse = filteredPosses.get(0); - if (lockType.equals(foundPosse.getTaskLock().getType())) { + final TaskLockPosse foundPosse = reusablePosses.get(0); + if (request.getType().equals(foundPosse.getTaskLock().getLockType()) && + request.getGranularity() == foundPosse.getTaskLock().getGranularity()) { return foundPosse; } else { - throw new ISE( - "Task[%s] already acquired a lock for interval[%s] but different type[%s]", - taskId, - interval, - foundPosse.getTaskLock().getType() - ); + if (request.getType() != foundPosse.getTaskLock().getLockType()) { + throw new ISE( + "Task[%s] already acquired a lock for interval[%s] but different type[%s]", + taskId, + request.getInterval(), + foundPosse.getTaskLock().getLockType() + ); + } else { + throw new ISE( + "Task[%s] already acquired a lock for interval[%s] but different granularity[%s]", + taskId, + request.getInterval(), + foundPosse.getTaskLock().getGranularity() + ); + } } - } else { - // case 3) we found multiple lock posses for the given task - throw new ISE( - "Task group[%s] has multiple locks for the same interval[%s]?", - groupId, - interval - ); } } else { // We don't have any locks for dataSource and interval. // Let's make a new one. - return createNewTaskLockPosse( - lockType, - groupId, - dataSource, - interval, - preferredVersion, - priority, - revoked - ); + return createNewTaskLockPosse(request); } } finally { @@ -554,51 +500,19 @@ private TaskLockPosse createOrFindLockPosse( * previously assigned to the same interval. This invariant is only mostly guaranteed, however; we assume clock * monotonicity and that callers specifying {@code preferredVersion} are doing the right thing. * - * @param lockType lock type - * @param groupId group id of task - * @param dataSource data source of task - * @param interval interval to be locked - * @param preferredVersion preferred version string - * @param priority lock priority - * @param revoked indicate the lock is revoked + * @param request request to lock * * @return a new {@link TaskLockPosse} */ - private TaskLockPosse createNewTaskLockPosse( - TaskLockType lockType, - String groupId, - String dataSource, - Interval interval, - @Nullable String preferredVersion, - int priority, - boolean revoked - ) + // TODO: new class for pair? + private TaskLockPosse createNewTaskLockPosse(LockRequest request) { giant.lock(); try { - // Create new TaskLock and assign it a version. - // Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This - // may not always be true, unfortunately. See below.) - - final String version; - - if (preferredVersion != null) { - // We have a preferred version. We'll trust our caller to not break our ordering assumptions and just use it. - version = preferredVersion; - } else { - // We are running under an interval lock right now, so just using the current time works as long as we can - // trustour clock to be monotonic and have enough resolution since the last time we created a TaskLock for - // the same interval. This may not always be true; to assure it we would need to use some method of - // timekeeping other than the wall clock. - version = DateTimes.nowUtc().toString(); - } - - final TaskLockPosse posseToUse = new TaskLockPosse( - new TaskLock(lockType, groupId, dataSource, interval, version, priority, revoked) - ); - running.computeIfAbsent(dataSource, k -> new TreeMap<>()) - .computeIfAbsent(interval.getStart(), k -> new TreeMap<>(Comparators.intervalsByStartThenEnd())) - .computeIfAbsent(interval, k -> new ArrayList<>()) + final TaskLockPosse posseToUse = new TaskLockPosse(request.toLock()); + running.computeIfAbsent(request.getDataSource(), k -> new TreeMap<>()) + .computeIfAbsent(request.getInterval().getStart(), k -> new TreeMap<>(Comparators.intervalsByStartThenEnd())) + .computeIfAbsent(request.getInterval(), k -> new ArrayList<>()) .add(posseToUse); return posseToUse; @@ -608,6 +522,19 @@ private TaskLockPosse createNewTaskLockPosse( } } + private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment request) + { + return metadataStorageCoordinator.allocatePendingSegment( + request.getDataSource(), + request.getSequenceName(), + request.getPrevisousSegmentId(), + request.getInterval(), + request.getShardSpecFactory(), + request.getVersion(), + request.isSkipSegmentLineageCheck() + ); + } + /** * Perform the given action with a guarantee that the locks of the task are not revoked in the middle of action. This * method first checks that all locks for the given task and intervals are valid and perform the right action. @@ -615,32 +542,49 @@ private TaskLockPosse createNewTaskLockPosse( * The given action should be finished as soon as possible because all other methods in this class are blocked until * this method is finished. * - * @param task task performing a critical action - * @param intervals intervals - * @param action action to be performed inside of the critical section + * @param task task performing a critical action + * @param intervalToPartitionIds partitionIds which should be locked by task + * @param action action to be performed inside of the critical section */ - public T doInCriticalSection(Task task, List intervals, CriticalAction action) throws Exception + public T doInCriticalSection( + Task task, + Map> intervalToPartitionIds, + CriticalAction action + ) throws Exception { giant.lock(); + // TODO: reduce contention by checking dataSource and interval. + // TODO: also cache taskLocks + try { - return action.perform(isTaskLocksValid(task, intervals)); + return action.perform(isTaskLocksValid(task, intervalToPartitionIds)); } finally { giant.unlock(); } } - private boolean isTaskLocksValid(Task task, List intervals) + private boolean isTaskLocksValid(Task task, Map> intervalToPartitionIds) { giant.lock(); try { - return intervals + return intervalToPartitionIds + .entrySet() .stream() - .allMatch(interval -> { - final TaskLock lock = getOnlyTaskLockPosseContainingInterval(task, interval).getTaskLock(); + .allMatch(entry -> { + // TODO: segment lock validation + // needs to check used segments, valid segment locks for used segments, + final List lockPosses = getOnlyTaskLockPosseContainingInterval( + task, + entry.getKey(), + entry.getValue() + ); // Tasks cannot enter the critical section with a shared lock - return !lock.isRevoked() && lock.getType() != TaskLockType.SHARED; + final boolean allLocksAreValid = lockPosses.stream().allMatch( + posse -> !posse.getTaskLock().isRevoked() && posse.getTaskLock().getLockType() != TaskLockType.SHARED + ); + return allLocksAreValid && lockPosses.size() == entry.getValue().size(); }); } finally { @@ -735,6 +679,11 @@ public TaskLock apply(TaskLockPosse taskLockPosse) } } + public void unlock(final Task task, final Interval interval) + { + unlock(task, interval, null); + } + /** * Release lock held for a task on a particular interval. Does nothing if the task does not currently * hold the mentioned lock. @@ -742,7 +691,7 @@ public TaskLock apply(TaskLockPosse taskLockPosse) * @param task task to unlock * @param interval interval to unlock */ - public void unlock(final Task task, final Interval interval) + public void unlock(final Task task, final Interval interval, @Nullable Integer partitionId) { giant.lock(); @@ -768,12 +717,34 @@ public void unlock(final Task task, final Interval interval) final List posses = possesHolder.stream() .filter(posse -> posse.containsTask(task)) .collect(Collectors.toList()); - for (TaskLockPosse taskLockPosse : posses) { final TaskLock taskLock = taskLockPosse.getTaskLock(); + if (taskLock instanceof TimeChunkLock) { + if (partitionId != null) { + throw new ISE( + "PartitoinId[%s] are set, but lock granularity is timeChunk for task[%s] and interval[%s]", + partitionId, + task, + interval + ); + } + } else if (taskLock instanceof SegmentLock) { + if (partitionId == null) { + throw new ISE( + "PartitoinId are missing, but lock granularity is segmentLock for task[%s] and interval[%s]", + task, + interval + ); + } + + if (((SegmentLock) taskLock).getPartitionId() != partitionId) { + continue; + } + } + // Remove task from live list - log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock.getGroupId()); + log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock); final boolean removed = taskLockPosse.removeTask(task); if (taskLockPosse.isTasksEmpty()) { @@ -846,7 +817,13 @@ public void remove(final Task task) try { log.info("Removing task[%s] from activeTasks", task.getId()); for (final TaskLockPosse taskLockPosse : findLockPossesForTask(task)) { - unlock(task, taskLockPosse.getTaskLock().getInterval()); + unlock( + task, + taskLockPosse.getTaskLock().getInterval(), + taskLockPosse.getTaskLock().getGranularity() == LockGranularity.SEGMENT + ? ((SegmentLock) taskLockPosse.taskLock).getPartitionId() + : null + ); } } finally { @@ -939,10 +916,21 @@ private List findLockPossesOverlapsInterval(final String dataSour } @VisibleForTesting - TaskLockPosse getOnlyTaskLockPosseContainingInterval(Task task, Interval interval) + List getOnlyTaskLockPosseContainingInterval(Task task, Interval interval) { giant.lock(); + try { + return getOnlyTaskLockPosseContainingInterval(task, interval, Collections.emptyList()); + } + finally { + giant.unlock(); + } + } + @VisibleForTesting + List getOnlyTaskLockPosseContainingInterval(Task task, Interval interval, List partitionIds) + { + giant.lock(); try { final List filteredPosses = findLockPossesContainingInterval(task.getDataSource(), interval) .stream() @@ -952,9 +940,33 @@ TaskLockPosse getOnlyTaskLockPosseContainingInterval(Task task, Interval interva if (filteredPosses.isEmpty()) { throw new ISE("Cannot find locks for task[%s] and interval[%s]", task.getId(), interval); } else if (filteredPosses.size() > 1) { - throw new ISE("There are multiple lockPosses for task[%s] and interval[%s]?", task.getId(), interval); + if (filteredPosses.stream() + .anyMatch(posse -> posse.getTaskLock().getGranularity() == LockGranularity.TIME_CHUNK)) { + throw new ISE( + "There are multiple timeChunk lockPosses for task[%s] and interval[%s]?", + task.getId(), + interval + ); + } else { + final Map partitionIdsOfLocks = new HashMap<>(); + for (TaskLockPosse posse : filteredPosses) { + final SegmentLock segmentLock = (SegmentLock) posse.getTaskLock(); + partitionIdsOfLocks.put(segmentLock.getPartitionId(), posse); + } + + if (partitionIds.stream().allMatch(partitionIdsOfLocks::containsKey)) { + return partitionIds.stream().map(partitionIdsOfLocks::get).collect(Collectors.toList()); + } else { + throw new ISE( + "Task[%s] doesn't have locks for interval[%s] partitions[%]", + task.getId(), + interval, + partitionIds.stream().filter(pid -> !partitionIdsOfLocks.containsKey(pid)).collect(Collectors.toList()) + ); + } + } } else { - return filteredPosses.get(0); + return filteredPosses; } } finally { @@ -962,16 +974,22 @@ TaskLockPosse getOnlyTaskLockPosseContainingInterval(Task task, Interval interva } } - private static boolean matchGroupIdAndContainInterval(TaskLock existingLock, String taskGroupId, Interval interval) + @VisibleForTesting + Set getActiveTasks() + { + return activeTasks; + } + + @VisibleForTesting + Map>>> getAllLocks() { - return existingLock.getInterval().contains(interval) && - existingLock.getGroupId().equals(taskGroupId); + return running; } private static boolean isAllSharedLocks(List lockPosses) { return lockPosses.stream() - .allMatch(taskLockPosse -> taskLockPosse.getTaskLock().getType().equals(TaskLockType.SHARED)); + .allMatch(taskLockPosse -> taskLockPosse.getTaskLock().getLockType().equals(TaskLockType.SHARED)); } private static boolean isAllRevocable(List lockPosses, int tryLockPriority) @@ -985,18 +1003,6 @@ private static boolean isRevocable(TaskLockPosse lockPosse, int tryLockPriority) return existingLock.isRevoked() || existingLock.getNonNullPriority() < tryLockPriority; } - @VisibleForTesting - Set getActiveTasks() - { - return activeTasks; - } - - @VisibleForTesting - Map>>> getAllLocks() - { - return running; - } - static class TaskLockPosse { private final TaskLock taskLock; @@ -1026,13 +1032,15 @@ TaskLock getTaskLock() boolean addTask(Task task) { - Preconditions.checkArgument( - taskLock.getGroupId().equals(task.getGroupId()), - "groupId[%s] of task[%s] is different from the existing lockPosse's groupId[%s]", - task.getGroupId(), - task.getId(), - taskLock.getGroupId() - ); + if (taskLock.getLockType() == TaskLockType.EXCLUSIVE) { + Preconditions.checkArgument( + taskLock.getGroupId().equals(task.getGroupId()), + "groupId[%s] of task[%s] is different from the existing lockPosse's groupId[%s]", + task.getGroupId(), + task.getId(), + taskLock.getGroupId() + ); + } Preconditions.checkArgument( taskLock.getNonNullPriority() == task.getPriority(), "priority[%s] of task[%s] is different from the existing lockPosse's priority[%s]", @@ -1060,6 +1068,36 @@ boolean isTasksEmpty() return taskIds.isEmpty(); } + boolean reusableFor(LockRequest request) + { + if (taskLock.getLockType() == request.getType() && taskLock.getGranularity() == request.getGranularity()) { + switch (taskLock.getLockType()) { + case SHARED: + // All shared lock is not reusable. Instead, a new lock posse is created for all lock request. + // See createOrFindLockPosse(). + return false; + case EXCLUSIVE: + if (request instanceof TimeChunkLockRequest) { + return taskLock.getInterval().contains(request.getInterval()) + && taskLock.getGroupId().equals(request.getGroupId()); + } else if (request instanceof SpecificSegmentLockRequest) { + final SegmentLock segmentLock = (SegmentLock) taskLock; + final SpecificSegmentLockRequest specificSegmentLockRequest = (SpecificSegmentLockRequest) request; + return segmentLock.getInterval().contains(specificSegmentLockRequest.getInterval()) + && segmentLock.getGroupId().equals(specificSegmentLockRequest.getGroupId()) + && specificSegmentLockRequest.getPartitionId() == segmentLock.getPartitionId(); + } else { + throw new ISE("Unknown request type[%s]", request); + } + //noinspection SuspiciousIndentAfterControlStatement + default: + throw new ISE("Unknown lock type[%s]", taskLock.getLockType()); + } + } + + return false; + } + void forEachTask(Consumer action) { Preconditions.checkNotNull(action, "action"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java new file mode 100644 index 000000000000..5ee851ebc00b --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java @@ -0,0 +1,147 @@ +/* + * 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.overlord; + +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.TimeChunkLock; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.DateTimes; +import org.joda.time.Interval; + +import javax.annotation.Nullable; + +public class TimeChunkLockRequest implements LockRequest +{ + private final TaskLockType lockType; + private final String groupId; + private final String dataSource; + private final Interval interval; + @Nullable + private final String preferredVersion; + private final int priority; + private final boolean revoked; + + public TimeChunkLockRequest( + TaskLockType lockType, + Task task, + Interval interval, + @Nullable String preferredVersion + ) + { + this(lockType, task.getGroupId(), task.getDataSource(), interval, preferredVersion, task.getPriority(), false); + } + + public TimeChunkLockRequest( + TaskLockType lockType, + String groupId, + String dataSource, + Interval interval, + @Nullable String preferredVersion, + int priority, + boolean revoked + ) + { + this.lockType = lockType; + this.groupId = groupId; + this.dataSource = dataSource; + this.interval = interval; + this.preferredVersion = preferredVersion; + this.priority = priority; + this.revoked = revoked; + } + + @Override + public LockGranularity getGranularity() + { + return LockGranularity.TIME_CHUNK; + } + + @Override + public TaskLockType getType() + { + return lockType; + } + + @Override + public String getGroupId() + { + return groupId; + } + + @Override + public String getDataSource() + { + return dataSource; + } + + @Override + public Interval getInterval() + { + return interval; + } + + @Override + public String getVersion() + { + return preferredVersion == null ? DateTimes.nowUtc().toString() : preferredVersion; + } + + @Override + public int getPriority() + { + return priority; + } + + @Override + public boolean isRevoked() + { + return revoked; + } + + @Override + public TaskLock toLock() + { + return new TimeChunkLock( + lockType, + groupId, + dataSource, + interval, + getVersion(), + priority, + revoked + ); + } + + @Override + public String toString() + { + return "TimeChunkLockRequest{" + + "lockType=" + lockType + + ", groupId='" + groupId + '\'' + + ", dataSource='" + dataSource + '\'' + + ", interval=" + interval + + ", preferredVersion='" + preferredVersion + '\'' + + ", priority=" + priority + + ", revoked=" + revoked + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 833cc172787a..69a17045b78d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -41,6 +41,7 @@ import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.NoopQueryRunner; @@ -54,10 +55,15 @@ import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.utils.CircularBuffer; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; import java.util.Map; @@ -230,7 +236,8 @@ public StreamAppenderatorDriver newDriver( schema.getGranularitySpec().getSegmentGranularity(), sequenceName, previousSegmentId, - skipSegmentLineageCheck + skipSegmentLineageCheck, + NumberedShardSpecFactory.instance() ) ), toolbox.getSegmentHandoffNotifierFactory(), @@ -292,4 +299,28 @@ public SeekableStreamIndexTaskRunner getRun return runner; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegment) + { + return false; + } + + @Override + @Nullable + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index c07118a99fd1..87b7eb1f4b87 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -46,11 +46,13 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +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.CheckPointDataSourceMetadataAction; import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; +import org.apache.druid.indexing.common.actions.SegmentLockAquireAction; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskUtils; @@ -383,7 +385,24 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception driver = task.newDriver(appenderator, toolbox, fireDepartmentMetrics); // Start up, set up initial sequences. - final Object restoredMetadata = driver.startJob(); + final Object restoredMetadata = driver.startJob( + segmentId -> { + try { + return toolbox.getTaskActionClient().submit( + new SegmentLockAquireAction( + TaskLockType.EXCLUSIVE, + segmentId.getInterval(), + segmentId.getVersion(), + segmentId.getShardSpec().getPartitionNum(), + 1000L + ) + ).isOk(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + ); if (restoredMetadata == null) { // no persist has happened so far // so either this is a brand new task or replacement of a failed task diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java new file mode 100644 index 000000000000..c611689ada5c --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java @@ -0,0 +1,94 @@ +/* + * 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.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class TaskLockTest +{ + private final ObjectMapper objectMapper = new DefaultObjectMapper(); + + @Test + public void testSerdeTimeChunkLock() throws IOException + { + final TimeChunkLock lock = new TimeChunkLock( + TaskLockType.EXCLUSIVE, + "groupId", + "dataSource", + Intervals.of("2019/2020"), + "version", + 100 + ); + final String json = objectMapper.writeValueAsString(lock); + final TaskLock fromJson = objectMapper.readValue(json, TaskLock.class); + + Assert.assertEquals(lock, fromJson); + } + + @Test + public void testDeserializeTimeChunkLockWithoutType() throws IOException + { + final TimeChunkLock expected = new TimeChunkLock( + TaskLockType.EXCLUSIVE, + "groupId", + "dataSource", + Intervals.of("2019/2020"), + "version", + 100 + ); + + final String json = "{\n" + + " \"lockType\" : \"EXCLUSIVE\",\n" + + " \"groupId\" : \"groupId\",\n" + + " \"dataSource\" : \"dataSource\",\n" + + " \"interval\" : \"2019-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z\",\n" + + " \"version\" : \"version\",\n" + + " \"priority\" : 100,\n" + + " \"revoked\" : false,\n" + + " \"type\" : \"timeChunk\"\n" + + "}"; + + Assert.assertEquals(expected, objectMapper.readValue(json, TaskLock.class)); + } + + @Test + public void testSerdeSegmentLock() throws IOException + { + final SegmentLock lock = new SegmentLock( + TaskLockType.EXCLUSIVE, + "groupId", + "dataSource", + Intervals.of("2019/2020"), + "version", + 0, + 100 + ); + final String json = objectMapper.writeValueAsString(lock); + final TaskLock fromJson = objectMapper.readValue(json, TaskLock.class); + + Assert.assertEquals(lock, fromJson); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java index 7879d8753569..8305433e86ac 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java @@ -27,6 +27,13 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; public class TestTasks { @@ -68,6 +75,31 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) { @@ -96,6 +128,31 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java index ec28edfb455c..1809dd15216f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java @@ -25,6 +25,7 @@ import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.jackson.DefaultObjectMapper; @@ -74,7 +75,7 @@ public void testSubmitSimple() throws Exception // return status code 200 and a list with size equals 1 Map responseBody = new HashMap(); - final List expectedLocks = Collections.singletonList(new TaskLock( + final List expectedLocks = Collections.singletonList(new TimeChunkLock( TaskLockType.SHARED, "groupId", "dataSource", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 1debc77f6b45..0fcf1b5af04e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -20,25 +20,29 @@ package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicate; -import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; +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.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.LinearShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpecFactory; +import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -48,6 +52,13 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Collectors; + public class SegmentAllocateActionTest { @Rule @@ -136,37 +147,40 @@ public void testManySegmentsSameInterval() id2.toString() ); - final TaskLock partyLock = Iterables.getOnlyElement( - FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter(input -> input.getInterval().contains(PARTY_TIME)) - ); + final List partyTimeLocks = taskActionTestKit.getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(PARTY_TIME)) + .collect(Collectors.toList()); + + Assert.assertEquals(3, partyTimeLocks.size()); assertSameIdentifier( - id1, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), + id1.getVersion(), new NumberedShardSpec(0, 0) - ) + ), + id1 ); assertSameIdentifier( - id2, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), + id1.getVersion(), new NumberedShardSpec(1, 0) - ) + ), + id2 ); assertSameIdentifier( - id3, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), + id1.getVersion(), new NumberedShardSpec(2, 0) - ) + ), + id3 ); } @@ -177,6 +191,8 @@ public void testResumeSequence() taskActionTestKit.getTaskLockbox().add(task); + final Map allocatedPartyTimeIds = new HashMap<>(); + final Map allocatedFutureIds = new HashMap<>(); final SegmentIdWithShardSpec id1 = allocate( task, PARTY_TIME, @@ -185,6 +201,8 @@ public void testResumeSequence() "s1", null ); + Assert.assertNotNull(id1); + allocatedPartyTimeIds.put(id1.getShardSpec().getPartitionNum(), id1); final SegmentIdWithShardSpec id2 = allocate( task, THE_DISTANT_FUTURE, @@ -193,6 +211,8 @@ public void testResumeSequence() "s1", id1.toString() ); + Assert.assertNotNull(id2); + allocatedFutureIds.put(id2.getShardSpec().getPartitionNum(), id2); final SegmentIdWithShardSpec id3 = allocate( task, PARTY_TIME, @@ -201,6 +221,8 @@ public void testResumeSequence() "s1", id2.toString() ); + Assert.assertNotNull(id3); + allocatedPartyTimeIds.put(id3.getShardSpec().getPartitionNum(), id3); final SegmentIdWithShardSpec id4 = allocate( task, PARTY_TIME, @@ -209,6 +231,7 @@ public void testResumeSequence() "s1", id1.toString() ); + Assert.assertNull(id4); final SegmentIdWithShardSpec id5 = allocate( task, THE_DISTANT_FUTURE, @@ -217,6 +240,8 @@ public void testResumeSequence() "s1", id1.toString() ); + Assert.assertNotNull(id5); + allocatedFutureIds.put(id5.getShardSpec().getPartitionNum(), id5); final SegmentIdWithShardSpec id6 = allocate( task, THE_DISTANT_FUTURE, @@ -225,6 +250,7 @@ public void testResumeSequence() "s1", id1.toString() ); + Assert.assertNull(id6); final SegmentIdWithShardSpec id7 = allocate( task, THE_DISTANT_FUTURE, @@ -233,65 +259,66 @@ public void testResumeSequence() "s1", id1.toString() ); + Assert.assertNotNull(id7); + allocatedFutureIds.put(id7.getShardSpec().getPartitionNum(), id7); + + final List partyLocks = taskActionTestKit.getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(PARTY_TIME)) + .collect(Collectors.toList()); + + Assert.assertEquals(2, partyLocks.size()); + final Map partitionIdToLock = new HashMap<>(); + partyLocks.forEach(lock -> { + Assert.assertEquals(LockGranularity.SEGMENT, lock.getGranularity()); + final SegmentLock segmentLock = (SegmentLock) lock; + partitionIdToLock.put(segmentLock.getPartitionId(), segmentLock); + }); + + for (Entry entry : partitionIdToLock.entrySet()) { + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + allocatedPartyTimeIds.get(entry.getKey()).getVersion(), + new NumberedShardSpec(entry.getValue().getPartitionId(), 0) + ), + allocatedPartyTimeIds.get(entry.getKey()) + ); + } - final TaskLock partyLock = Iterables.getOnlyElement( - FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(PARTY_TIME); - } - } - ) - ); - final TaskLock futureLock = Iterables.getOnlyElement( - FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(THE_DISTANT_FUTURE); - } - } - ) - ); + final List futureLocks = taskActionTestKit + .getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) + .collect(Collectors.toList()); + + Assert.assertEquals(1, futureLocks.size()); + partitionIdToLock.clear(); + futureLocks.forEach(lock -> { + Assert.assertEquals(LockGranularity.SEGMENT, lock.getGranularity()); + final SegmentLock segmentLock = (SegmentLock) lock; + partitionIdToLock.put(segmentLock.getPartitionId(), segmentLock); + }); + + for (Entry entry : partitionIdToLock.entrySet()) { + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(THE_DISTANT_FUTURE), + allocatedFutureIds.get(entry.getKey()).getVersion(), + new NumberedShardSpec(entry.getValue().getPartitionId(), 0) + ), + allocatedFutureIds.get(entry.getKey()) + ); + } - assertSameIdentifier( - id1, - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), - new NumberedShardSpec(0, 0) - ) - ); - assertSameIdentifier( - id2, - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(THE_DISTANT_FUTURE), - futureLock.getVersion(), - new NumberedShardSpec(0, 0) - ) - ); - assertSameIdentifier( - id3, - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), - new NumberedShardSpec(1, 0) - ) - ); Assert.assertNull(id4); - assertSameIdentifier(id5, id2); + assertSameIdentifier(id2, id5); Assert.assertNull(id6); - assertSameIdentifier(id7, id2); + assertSameIdentifier(id2, id7); } @Test @@ -327,84 +354,80 @@ public void testMultipleSequences() "s2", id2.toString() ); - final SegmentIdWithShardSpec id6 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null); - - final TaskLock partyLock = Iterables.getOnlyElement( - FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(PARTY_TIME); - } - } - ) - ); - final TaskLock futureLock = Iterables.getOnlyElement( - FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) - .filter( - new Predicate() - { - @Override - public boolean apply(TaskLock input) - { - return input.getInterval().contains(THE_DISTANT_FUTURE); - } - } - ) + final SegmentIdWithShardSpec id6 = allocate( + task, + PARTY_TIME, + Granularities.NONE, + Granularities.HOUR, + "s1", + null ); + final List partyLocks = taskActionTestKit.getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(PARTY_TIME)) + .collect(Collectors.toList()); + + Assert.assertEquals(3, partyLocks.size()); + assertSameIdentifier( - id1, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), + partyLocks.get(0).getVersion(), new NumberedShardSpec(0, 0) - ) + ), + id1 ); assertSameIdentifier( - id2, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), + partyLocks.get(1).getVersion(), new NumberedShardSpec(1, 0) - ) + ), + id2 ); assertSameIdentifier( - id3, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), - partyLock.getVersion(), + partyLocks.get(2).getVersion(), new NumberedShardSpec(2, 0) - ) + ), + id3 ); + + final List futureLocks = taskActionTestKit + .getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) + .collect(Collectors.toList()); + + Assert.assertEquals(2, futureLocks.size()); + assertSameIdentifier( - id4, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(THE_DISTANT_FUTURE), - futureLock.getVersion(), + futureLocks.get(0).getVersion(), new NumberedShardSpec(0, 0) - ) + ), + id4 ); assertSameIdentifier( - id5, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(THE_DISTANT_FUTURE), - futureLock.getVersion(), + futureLocks.get(1).getVersion(), new NumberedShardSpec(1, 0) - ) - ); - assertSameIdentifier( - id6, - id1 + ), + id5 ); + + assertSameIdentifier(id1, id6); } @Test @@ -437,7 +460,8 @@ public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception Granularities.NONE, Granularities.HOUR, "s1", - null + null, + LinearShardSpecFactory.instance() ); final SegmentIdWithShardSpec id2 = allocate( task, @@ -445,26 +469,27 @@ public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception Granularities.NONE, Granularities.HOUR, "s1", - id1.toString() + id1.toString(), + LinearShardSpecFactory.instance() ); assertSameIdentifier( - id1, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), PARTY_TIME.toString(), new LinearShardSpec(2) - ) + ), + id1 ); assertSameIdentifier( - id2, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), PARTY_TIME.toString(), new LinearShardSpec(3) - ) + ), + id2 ); } @@ -510,22 +535,22 @@ public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exceptio ); assertSameIdentifier( - id1, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), PARTY_TIME.toString(), new NumberedShardSpec(2, 2) - ) + ), + id1 ); assertSameIdentifier( - id2, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), PARTY_TIME.toString(), new NumberedShardSpec(3, 2) - ) + ), + id2 ); } @@ -556,13 +581,13 @@ public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() thr final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.DAY, "s1", null); assertSameIdentifier( - id1, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), PARTY_TIME.toString(), new NumberedShardSpec(2, 2) - ) + ), + id1 ); } @@ -593,13 +618,13 @@ public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throw final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.MINUTE, "s1", null); assertSameIdentifier( - id1, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), PARTY_TIME.toString(), new NumberedShardSpec(2, 2) - ) + ), + id1 ); } @@ -675,6 +700,9 @@ public void testCannotAddToExistingSingleDimensionShardSpecs() throws Exception @Test public void testSerde() throws Exception { + final ObjectMapper objectMapper = new DefaultObjectMapper(); + objectMapper.registerSubtypes(NumberedShardSpecFactory.class); + final SegmentAllocateAction action = new SegmentAllocateAction( DATA_SOURCE, PARTY_TIME, @@ -682,21 +710,69 @@ public void testSerde() throws Exception Granularities.HOUR, "s1", "prev", - false + false, + NumberedShardSpecFactory.instance() ); - final ObjectMapper objectMapper = new DefaultObjectMapper(); final SegmentAllocateAction action2 = (SegmentAllocateAction) objectMapper.readValue( objectMapper.writeValueAsBytes(action), TaskAction.class ); - Assert.assertEquals(DATA_SOURCE, action2.getDataSource()); - Assert.assertEquals(PARTY_TIME, action2.getTimestamp()); - Assert.assertEquals(Granularities.MINUTE, action2.getQueryGranularity()); - Assert.assertEquals(Granularities.HOUR, action2.getPreferredSegmentGranularity()); - Assert.assertEquals("s1", action2.getSequenceName()); - Assert.assertEquals("prev", action2.getPreviousSegmentId()); + Assert.assertEquals(action.getDataSource(), action2.getDataSource()); + Assert.assertEquals(action.getTimestamp(), action2.getTimestamp()); + Assert.assertEquals(action.getQueryGranularity(), action2.getQueryGranularity()); + Assert.assertEquals(action.getPreferredSegmentGranularity(), action2.getPreferredSegmentGranularity()); + Assert.assertEquals(action.getSequenceName(), action2.getSequenceName()); + Assert.assertEquals(action.getPreviousSegmentId(), action2.getPreviousSegmentId()); + Assert.assertEquals(action.isSkipSegmentLineageCheck(), action2.isSkipSegmentLineageCheck()); + } + + @Test + public void testWithShardSpecFactoryAndOvershadowingSegments() throws IOException + { + final Task task = NoopTask.create(); + taskActionTestKit.getTaskLockbox().add(task); + + final ObjectMapper objectMapper = new DefaultObjectMapper(); + + taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( + ImmutableSet.of( + DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(Granularities.HOUR.bucket(PARTY_TIME)) + .version(PARTY_TIME.toString()) + .shardSpec(new HashBasedNumberedShardSpec(0, 2, ImmutableList.of("dim1"), objectMapper)) + .build(), + DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(Granularities.HOUR.bucket(PARTY_TIME)) + .version(PARTY_TIME.toString()) + .shardSpec(new HashBasedNumberedShardSpec(1, 2, ImmutableList.of("dim1"), objectMapper)) + .build() + ) + ); + + final SegmentAllocateAction action = new SegmentAllocateAction( + DATA_SOURCE, + PARTY_TIME, + Granularities.MINUTE, + Granularities.HOUR, + "seq", + null, + true, + new HashBasedNumberedShardSpecFactory(ImmutableList.of("dim1"), 2) + ); + final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox()); + Assert.assertNotNull(segmentIdentifier); + + final ShardSpec shardSpec = segmentIdentifier.getShardSpec(); + Assert.assertEquals(2, shardSpec.getPartitionNum()); + + Assert.assertTrue(shardSpec instanceof HashBasedNumberedShardSpec); + final HashBasedNumberedShardSpec hashBasedNumberedShardSpec = (HashBasedNumberedShardSpec) shardSpec; + Assert.assertEquals(2, hashBasedNumberedShardSpec.getPartitions()); + Assert.assertEquals(ImmutableList.of("dim1"), hashBasedNumberedShardSpec.getPartitionDimensions()); } private SegmentIdWithShardSpec allocate( @@ -707,6 +783,27 @@ private SegmentIdWithShardSpec allocate( final String sequenceName, final String sequencePreviousId ) + { + return allocate( + task, + timestamp, + queryGranularity, + preferredSegmentGranularity, + sequenceName, + sequencePreviousId, + NumberedShardSpecFactory.instance() + ); + } + + private SegmentIdWithShardSpec allocate( + final Task task, + final DateTime timestamp, + final Granularity queryGranularity, + final Granularity preferredSegmentGranularity, + final String sequenceName, + final String sequencePreviousId, + final ShardSpecFactory shardSpecFactory + ) { final SegmentAllocateAction action = new SegmentAllocateAction( DATA_SOURCE, @@ -715,31 +812,27 @@ private SegmentIdWithShardSpec allocate( preferredSegmentGranularity, sequenceName, sequencePreviousId, - false + false, + shardSpecFactory ); return action.perform(task, taskActionTestKit.getTaskActionToolbox()); } - private void assertSameIdentifier(final SegmentIdWithShardSpec one, final SegmentIdWithShardSpec other) + private void assertSameIdentifier(final SegmentIdWithShardSpec expected, final SegmentIdWithShardSpec actual) { - Assert.assertEquals(one, other); - Assert.assertEquals(one.getShardSpec().getPartitionNum(), other.getShardSpec().getPartitionNum()); + Assert.assertEquals(expected, actual); + Assert.assertEquals(expected.getShardSpec().getPartitionNum(), actual.getShardSpec().getPartitionNum()); + Assert.assertEquals(expected.getShardSpec().getClass(), actual.getShardSpec().getClass()); - if (one.getShardSpec().getClass() == NumberedShardSpec.class - && other.getShardSpec().getClass() == NumberedShardSpec.class) { + if (expected.getShardSpec().getClass() == NumberedShardSpec.class + && actual.getShardSpec().getClass() == NumberedShardSpec.class) { Assert.assertEquals( - ((NumberedShardSpec) one.getShardSpec()).getPartitions(), - ((NumberedShardSpec) other.getShardSpec()).getPartitions() + ((NumberedShardSpec) expected.getShardSpec()).getPartitions(), + ((NumberedShardSpec) actual.getShardSpec()).getPartitions() ); - } else if (one.getShardSpec().getClass() == LinearShardSpec.class - && other.getShardSpec().getClass() == LinearShardSpec.class) { + } else if (expected.getShardSpec().getClass() == LinearShardSpec.class + && actual.getShardSpec().getClass() == LinearShardSpec.class) { // do nothing - } else { - throw new ISE( - "Unexpected shardSpecs [%s] and [%s]", - one.getShardSpec().getClass(), - other.getShardSpec().getClass() - ); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java index abd43d986ed7..c1b44c71d161 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -26,6 +26,8 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; +import org.apache.druid.indexing.overlord.LockResult; +import org.apache.druid.indexing.overlord.TimeChunkLockRequest; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -88,16 +90,22 @@ public class SegmentInsertActionTest 1024 ); + private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval, long timeoutMs) + throws InterruptedException + { + return actionTestKit.getTaskLockbox().lock(task, new TimeChunkLockRequest(lockType, task, interval, null), timeoutMs); + } + @Test public void testSimple() throws Exception { final Task task = new NoopTask(null, null, 0, 0, null, null, null); final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2)); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); + acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); actionTestKit.getTaskLockbox().doInCriticalSection( task, - Collections.singletonList(INTERVAL), + Collections.singletonMap(INTERVAL, Collections.emptyList()), CriticalAction.builder() .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) .onInvalidLocks( @@ -124,13 +132,13 @@ public void testFailBadVersion() throws Exception final Task task = new NoopTask(null, null, 0, 0, null, null, null); final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); + acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); thrown.expect(IllegalStateException.class); thrown.expectMessage(CoreMatchers.containsString("are not covered by locks")); final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection( task, - Collections.singletonList(INTERVAL), + Collections.singletonMap(INTERVAL, Collections.emptyList()), CriticalAction.>builder() .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) .onInvalidLocks( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java index 463916f9567e..cfae5f65024b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java @@ -25,8 +25,10 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.indexing.overlord.ObjectMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.TimeChunkLockRequest; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -86,12 +88,18 @@ public class SegmentTransactionalInsertActionTest 1024 ); + private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval, long timeoutMs) + throws InterruptedException + { + return actionTestKit.getTaskLockbox().lock(task, new TimeChunkLockRequest(lockType, task, interval, null), timeoutMs); + } + @Test public void testTransactional() throws Exception { final Task task = new NoopTask(null, null, 0, 0, null, null, null); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); + acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); SegmentPublishResult result1 = new SegmentTransactionalInsertAction( ImmutableSet.of(SEGMENT1), @@ -132,7 +140,7 @@ public void testFailTransactional() throws Exception { final Task task = new NoopTask(null, null, 0, 0, null, null, null); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); + acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); SegmentPublishResult result = new SegmentTransactionalInsertAction( ImmutableSet.of(SEGMENT1), @@ -152,7 +160,7 @@ public void testFailBadVersion() throws Exception final Task task = new NoopTask(null, null, 0, 0, null, null, null); final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); - actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); + acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); thrown.expect(IllegalStateException.class); thrown.expectMessage(CoreMatchers.containsString("are not covered by locks")); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SurrogateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SurrogateActionTest.java index 31bc77392fc8..5edcbb579203 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SurrogateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SurrogateActionTest.java @@ -35,8 +35,9 @@ public class SurrogateActionTest public void testSerde() throws IOException { final ObjectMapper objectMapper = new DefaultObjectMapper(); - final SurrogateAction surrogateAction = new SurrogateAction<>( - "testId", new LockTryAcquireAction(TaskLockType.EXCLUSIVE, Intervals.of("2018-01-01/2019-01-01")) + final SurrogateAction surrogateAction = new SurrogateAction<>( + "testId", + new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, Intervals.of("2018-01-01/2019-01-01")) ); final String json = objectMapper.writeValueAsString(surrogateAction); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java index 175417cb845b..2d5a753e19c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java @@ -27,7 +27,10 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; +import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.TimeChunkLockRequest; +import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; @@ -52,7 +55,10 @@ public class TaskActionPreconditionsTest @Before public void setup() { - lockbox = new TaskLockbox(new HeapMemoryTaskStorage(new TaskStorageConfig(null))); + lockbox = new TaskLockbox( + new HeapMemoryTaskStorage(new TaskStorageConfig(null)), + new TestIndexerMetadataStorageCoordinator() + ); task = NoopTask.create(); lockbox.add(task); @@ -78,6 +84,11 @@ public void setup() ); } + private LockResult tryTimeChunkLock(TaskLockType lockType, Task task, Interval interval) + { + return lockbox.tryLock(task, new TimeChunkLockRequest(lockType, task, interval, null)); + } + @Test public void testCheckLockCoversSegments() { @@ -91,7 +102,7 @@ public void testCheckLockCoversSegments() Collectors.toMap( Function.identity(), interval -> { - final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); Assert.assertNotNull(lock); return lock; } @@ -113,7 +124,7 @@ public void testCheckLargeLockCoversSegments() Collectors.toMap( Function.identity(), interval -> { - final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); Assert.assertNotNull(lock); return lock; } @@ -137,7 +148,7 @@ public void testCheckLockCoversSegmentsWithOverlappedIntervals() Collectors.toMap( Function.identity(), interval -> { - final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); Assert.assertNotNull(lock); return lock; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index ceb65d1f31e7..e1477d603743 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -75,7 +75,6 @@ public TaskActionToolbox getTaskActionToolbox() public void before() { taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(new Period("PT24H"))); - taskLockbox = new TaskLockbox(taskStorage); testDerbyConnector = new TestDerbyConnector( Suppliers.ofInstance(new MetadataStorageConnectorConfig()), Suppliers.ofInstance(metadataStorageTablesConfig) @@ -86,6 +85,7 @@ public void before() metadataStorageTablesConfig, testDerbyConnector ); + taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); metadataSegmentManager = new SQLMetadataSegmentManager( objectMapper, Suppliers.ofInstance(new MetadataSegmentManagerConfig()), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LockAcquireActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireActionTest.java similarity index 84% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LockAcquireActionTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireActionTest.java index 4f64f384840f..7caaa6361dfc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LockAcquireActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireActionTest.java @@ -32,7 +32,7 @@ import java.io.IOException; -public class LockAcquireActionTest +public class TimeChunkLockAcquireActionTest { @Rule public TaskActionTestKit actionTestKit = new TaskActionTestKit(); @@ -42,14 +42,14 @@ public class LockAcquireActionTest @Test public void testSerdeWithAllFields() throws IOException { - final LockAcquireAction expected = new LockAcquireAction( + final TimeChunkLockAcquireAction expected = new TimeChunkLockAcquireAction( TaskLockType.SHARED, Intervals.of("2017-01-01/2017-01-02"), 1000 ); final byte[] bytes = mapper.writeValueAsBytes(expected); - final LockAcquireAction actual = mapper.readValue(bytes, LockAcquireAction.class); + final TimeChunkLockAcquireAction actual = mapper.readValue(bytes, TimeChunkLockAcquireAction.class); Assert.assertEquals(expected.getType(), actual.getType()); Assert.assertEquals(expected.getInterval(), actual.getInterval()); Assert.assertEquals(expected.getTimeoutMs(), actual.getTimeoutMs()); @@ -60,8 +60,8 @@ public void testSerdeFromJsonWithMissingFields() throws IOException { final String json = "{ \"type\": \"lockAcquire\", \"interval\" : \"2017-01-01/2017-01-02\" }"; - final LockAcquireAction actual = mapper.readValue(json, LockAcquireAction.class); - final LockAcquireAction expected = new LockAcquireAction( + final TimeChunkLockAcquireAction actual = mapper.readValue(json, TimeChunkLockAcquireAction.class); + final TimeChunkLockAcquireAction expected = new TimeChunkLockAcquireAction( TaskLockType.EXCLUSIVE, Intervals.of("2017-01-01/2017-01-02"), 0 @@ -75,7 +75,7 @@ public void testSerdeFromJsonWithMissingFields() throws IOException public void testWithLockType() { final Task task = NoopTask.create(); - final LockAcquireAction action = new LockAcquireAction( + final TimeChunkLockAcquireAction action = new TimeChunkLockAcquireAction( TaskLockType.EXCLUSIVE, Intervals.of("2017-01-01/2017-01-02"), 1000 @@ -90,7 +90,7 @@ public void testWithLockType() public void testWithoutLockType() { final Task task = NoopTask.create(); - final LockAcquireAction action = new LockAcquireAction( + final TimeChunkLockAcquireAction action = new TimeChunkLockAcquireAction( null, Intervals.of("2017-01-01/2017-01-02"), 1000 diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LockTryAcquireActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireActionTest.java similarity index 82% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LockTryAcquireActionTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireActionTest.java index 9b1077a9346a..c6b65da1e0ea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/LockTryAcquireActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireActionTest.java @@ -32,7 +32,7 @@ import java.io.IOException; -public class LockTryAcquireActionTest +public class TimeChunkLockTryAcquireActionTest { @Rule public TaskActionTestKit actionTestKit = new TaskActionTestKit(); @@ -42,13 +42,13 @@ public class LockTryAcquireActionTest @Test public void testSerdeWithAllFields() throws IOException { - final LockTryAcquireAction expected = new LockTryAcquireAction( + final TimeChunkLockTryAcquireAction expected = new TimeChunkLockTryAcquireAction( TaskLockType.SHARED, Intervals.of("2017-01-01/2017-01-02") ); final byte[] bytes = mapper.writeValueAsBytes(expected); - final LockTryAcquireAction actual = mapper.readValue(bytes, LockTryAcquireAction.class); + final TimeChunkLockTryAcquireAction actual = mapper.readValue(bytes, TimeChunkLockTryAcquireAction.class); Assert.assertEquals(expected.getType(), actual.getType()); Assert.assertEquals(expected.getInterval(), actual.getInterval()); } @@ -58,8 +58,8 @@ public void testSerdeFromJsonWithMissingFields() throws IOException { final String json = "{ \"type\": \"lockTryAcquire\", \"interval\" : \"2017-01-01/2017-01-02\" }"; - final LockTryAcquireAction actual = mapper.readValue(json, LockTryAcquireAction.class); - final LockTryAcquireAction expected = new LockTryAcquireAction( + final TimeChunkLockTryAcquireAction actual = mapper.readValue(json, TimeChunkLockTryAcquireAction.class); + final TimeChunkLockTryAcquireAction expected = new TimeChunkLockTryAcquireAction( TaskLockType.EXCLUSIVE, Intervals.of("2017-01-01/2017-01-02") ); @@ -71,7 +71,7 @@ public void testSerdeFromJsonWithMissingFields() throws IOException public void testWithLockType() { final Task task = NoopTask.create(); - final LockTryAcquireAction action = new LockTryAcquireAction( + final TimeChunkLockTryAcquireAction action = new TimeChunkLockTryAcquireAction( TaskLockType.EXCLUSIVE, Intervals.of("2017-01-01/2017-01-02") ); @@ -85,7 +85,7 @@ public void testWithLockType() public void testWithoutLockType() { final Task task = NoopTask.create(); - final LockTryAcquireAction action = new LockTryAcquireAction( + final TimeChunkLockTryAcquireAction action = new TimeChunkLockTryAcquireAction( null, Intervals.of("2017-01-01/2017-01-02") ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 3155ac9d4e87..e28ca22f7c99 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1370,7 +1370,6 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( final Long maxTotalRows ) { - ObjectMapper objectMapper = new DefaultObjectMapper(); DataSchema dataSchema = new DataSchema( "test_ds", TestHelper.makeJsonMapper().convertValue( @@ -1465,8 +1464,6 @@ private void awaitHandoffs() throws InterruptedException private void makeToolboxFactory(final File directory) { taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); - taskLockbox = new TaskLockbox(taskStorage); - publishedSegments = new CopyOnWriteArrayList<>(); ObjectMapper mapper = new DefaultObjectMapper(); @@ -1514,6 +1511,8 @@ public SegmentPublishResult announceHistoricalSegments( return result; } }; + + taskLockbox = new TaskLockbox(taskStorage, mdc); final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, true, null, null); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index ea109c8b6cdc..9d2e4e20caf5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -28,13 +28,13 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.actions.LocalTaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.CompactionTask.Builder; import org.apache.druid.java.util.common.ISE; @@ -43,7 +43,6 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; @@ -55,7 +54,9 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedOverwritingShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -65,18 +66,24 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; import java.io.BufferedWriter; import java.io.File; -import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; public class CompactionTaskRunTest extends IngestionTestBase { + public static final String DATA_SOURCE = "test"; + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -162,7 +169,7 @@ public void testRun() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedOverwritingShardSpec(32768, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); } } @@ -194,7 +201,7 @@ public void testRunCompactionTwiceWithoutKeepSegmentGranularity() throws Excepti List segments = resultPair.rhs; Assert.assertEquals(1, segments.size()); - Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); + Assert.assertEquals(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00"), segments.get(0).getInterval()); Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(0).getShardSpec()); final CompactionTask compactionTask2 = builder @@ -209,7 +216,7 @@ public void testRunCompactionTwiceWithoutKeepSegmentGranularity() throws Excepti segments = resultPair.rhs; Assert.assertEquals(1, segments.size()); - Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); + Assert.assertEquals(Intervals.of("2014-01-01T00:00:00/2014-01-01T03:00:00"), segments.get(0).getInterval()); Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(0).getShardSpec()); } @@ -243,7 +250,7 @@ public void testRunCompactionTwiceWithKeepSegmentGranularity() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); } final CompactionTask compactionTask2 = builder @@ -260,7 +267,92 @@ public void testRunCompactionTwiceWithKeepSegmentGranularity() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + 1, 0, 2, (short) 2, (short) 1), segments.get(i).getShardSpec()); + } + } + + @Test + public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exception + { + runIndexTask(); + + final Builder builder = new Builder( + DATA_SOURCE, + getObjectMapper(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null, + rowIngestionMetersFactory, + coordinatorClient, + segmentLoaderFactory, + retryPolicyFactory + ); + + final CompactionTask compactionTask = builder + .interval(Intervals.of("2014-01-01T00:00:00/2014-01-02T03:00:00")) + .build(); + + File tmpDir = temporaryFolder.newFolder(); + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("2014-01-01T03:00:10Z,a,1\n"); + writer.write("2014-01-01T03:00:10Z,b,2\n"); + writer.write("2014-01-01T03:00:10Z,c,3\n"); + writer.write("2014-01-01T04:00:20Z,a,1\n"); + writer.write("2014-01-01T04:00:20Z,b,2\n"); + writer.write("2014-01-01T04:00:20Z,c,3\n"); + writer.write("2014-01-01T05:00:30Z,a,1\n"); + writer.write("2014-01-01T05:00:30Z,b,2\n"); + writer.write("2014-01-01T05:00:30Z,c,3\n"); + } + + IndexTask indexTask = new IndexTask( + null, + null, + IndexTaskTest.createIngestionSpec( + getObjectMapper(), + tmpDir, + DEFAULT_PARSE_SPEC, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + null + ), + IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true), + false + ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null, + rowIngestionMetersFactory + ); + + final Future>> compactionFuture = exec.submit( + () -> runTask(compactionTask) + ); + + final Future>> indexFuture = exec.submit( + () -> runTask(indexTask) + ); + + Assert.assertTrue(indexFuture.get().lhs.isSuccess()); + + List segments = indexFuture.get().rhs; + Assert.assertEquals(6, segments.size()); + + for (int i = 0; i < 6; i++) { + Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", 3 + i / 2, 3 + i / 2 + 1), segments.get(i).getInterval()); + Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + } + + Assert.assertTrue(compactionFuture.get().lhs.isSuccess()); + + segments = compactionFuture.get().rhs; + Assert.assertEquals(3, segments.size()); + + for (int i = 0; i < 3; i++) { + Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); + Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); } } @@ -316,7 +408,163 @@ public void testWithSegmentGranularity() throws Exception } } + @Test + public void testCompactThenAppend() throws Exception + { + runIndexTask(); + + final Builder builder = new Builder( + DATA_SOURCE, + getObjectMapper(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null, + rowIngestionMetersFactory, + coordinatorClient, + segmentLoaderFactory, + retryPolicyFactory + ); + + final CompactionTask compactionTask = builder + .interval(Intervals.of("2014-01-01/2014-01-02")) + .build(); + + final Set expectedSegments = new HashSet<>(); + final Pair> compactionResult = runTask(compactionTask); + Assert.assertTrue(compactionResult.lhs.isSuccess()); + expectedSegments.addAll(compactionResult.rhs); + + final Pair> appendResult = runAppendTask(); + Assert.assertTrue(appendResult.lhs.isSuccess()); + expectedSegments.addAll(appendResult.rhs); + + final Set usedSegments = new HashSet<>( + getStorageCoordinator().getUsedSegmentsForIntervals( + DATA_SOURCE, + Collections.singletonList(Intervals.of("2014-01-01/2014-01-02")) + ) + ); + + Assert.assertEquals(expectedSegments, usedSegments); + } + + @Test + public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception + { + runIndexTask(); + + // make sure that indexTask becomes ready first, then compactionTask becomes ready, then indexTask runs + final CountDownLatch compactionTaskReadyLatch = new CountDownLatch(1); + final CountDownLatch indexTaskStartLatch = new CountDownLatch(1); + final Future>> indexFuture = exec.submit( + () -> runIndexTask(compactionTaskReadyLatch, indexTaskStartLatch, false) + ); + + final Builder builder = new Builder( + DATA_SOURCE, + getObjectMapper(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null, + rowIngestionMetersFactory, + coordinatorClient, + segmentLoaderFactory, + retryPolicyFactory + ); + + final CompactionTask compactionTask = builder + .interval(Intervals.of("2014-01-01T00:00:00/2014-01-02T03:00:00")) + .build(); + + final Future>> compactionFuture = exec.submit( + () -> { + compactionTaskReadyLatch.await(); + return runTask(compactionTask, indexTaskStartLatch, null); + } + ); + + Assert.assertTrue(indexFuture.get().lhs.isSuccess()); + + List segments = indexFuture.get().rhs; + Assert.assertEquals(6, segments.size()); + + for (int i = 0; i < 6; i++) { + Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), segments.get(i).getInterval()); + Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), segments.get(i).getShardSpec()); + } + + final Pair> compactionResult = compactionFuture.get(); + Assert.assertEquals(TaskState.FAILED, compactionResult.lhs.getStatusCode()); + } + + @Test + public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exception + { + runIndexTask(); + + final Builder builder = new Builder( + DATA_SOURCE, + getObjectMapper(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null, + rowIngestionMetersFactory, + coordinatorClient, + segmentLoaderFactory, + retryPolicyFactory + ); + + final CompactionTask compactionTask = builder + .interval(Intervals.of("2014-01-01T00:00:00/2014-01-02T03:00:00")) + .build(); + + // make sure that compactionTask becomes ready first, then the indexTask becomes ready, then compactionTask runs + final CountDownLatch indexTaskReadyLatch = new CountDownLatch(1); + final CountDownLatch compactionTaskStartLatch = new CountDownLatch(1); + final Future>> compactionFuture = exec.submit( + () -> { + final Pair> pair = runTask( + compactionTask, + indexTaskReadyLatch, + compactionTaskStartLatch + ); + return pair; + } + ); + + final Future>> indexFuture = exec.submit( + () -> { + indexTaskReadyLatch.await(); + return runIndexTask(compactionTaskStartLatch, null, false); + } + ); + + Assert.assertTrue(indexFuture.get().lhs.isSuccess()); + + List segments = indexFuture.get().rhs; + Assert.assertEquals(6, segments.size()); + + for (int i = 0; i < 6; i++) { + Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), segments.get(i).getInterval()); + Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), segments.get(i).getShardSpec()); + } + + final Pair> compactionResult = compactionFuture.get(); + Assert.assertEquals(TaskState.FAILED, compactionResult.lhs.getStatusCode()); + } + private Pair> runIndexTask() throws Exception + { + return runIndexTask(null, null, false); + } + + private Pair> runAppendTask() throws Exception + { + return runIndexTask(null, null, true); + } + + private Pair> runIndexTask( + @Nullable CountDownLatch readyLatchToCountDown, + @Nullable CountDownLatch latchToAwaitBeforeRun, + boolean appendToExisting + ) throws Exception { File tmpDir = temporaryFolder.newFolder(); File tmpFile = File.createTempFile("druid", "index", tmpDir); @@ -336,7 +584,8 @@ private Pair> runIndexTask() throws Exception IndexTask indexTask = new IndexTask( null, null, - createIngestionSpec( + IndexTaskTest.createIngestionSpec( + getObjectMapper(), tmpDir, DEFAULT_PARSE_SPEC, new UniformGranularitySpec( @@ -345,7 +594,7 @@ private Pair> runIndexTask() throws Exception null ), IndexTaskTest.createTuningConfig(2, 2, null, 2L, null, null, false, true), - false + appendToExisting ), null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, @@ -353,14 +602,23 @@ private Pair> runIndexTask() throws Exception rowIngestionMetersFactory ); - return runTask(indexTask); + return runTask(indexTask, readyLatchToCountDown, latchToAwaitBeforeRun); } private Pair> runTask(Task task) throws Exception + { + return runTask(task, null, null); + } + + private Pair> runTask( + Task task, + @Nullable CountDownLatch readyLatchToCountDown, + @Nullable CountDownLatch latchToAwaitBeforeRun + ) throws Exception { getLockbox().add(task); getTaskStorage().insert(task, TaskStatus.running(task.getId())); - final LocalTaskActionClient actionClient = createActionClient(task); + final TestLocalTaskActionClient actionClient = createActionClient(task); final File deepStorageDir = temporaryFolder.newFolder(); final ObjectMapper objectMapper = getObjectMapper(); @@ -369,26 +627,6 @@ private Pair> runTask(Task task) throws Exception ); objectMapper.registerSubtypes(LocalDataSegmentPuller.class); - final List segments = new ArrayList<>(); - final DataSegmentPusher pusher = new LocalDataSegmentPusher( - new LocalDataSegmentPusherConfig() - { - @Override - public File getStorageDirectory() - { - return deepStorageDir; - } - } - ) - { - @Override - public DataSegment push(File file, DataSegment segment, boolean useUniquePath) throws IOException - { - segments.add(segment); - return super.push(file, segment, useUniquePath); - } - }; - final SegmentLoader loader = new SegmentLoaderLocalCacheManager( getIndexIO(), new SegmentLoaderConfig() { @@ -414,7 +652,7 @@ public File getPath() null, actionClient, null, - pusher, + new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig()), new NoopDataSegmentKiller(), null, null, @@ -440,12 +678,19 @@ public File getPath() ); if (task.isReady(box.getTaskActionClient())) { + if (readyLatchToCountDown != null) { + readyLatchToCountDown.countDown(); + } + if (latchToAwaitBeforeRun != null) { + latchToAwaitBeforeRun.await(); + } TaskStatus status = task.run(box); shutdownTask(task); + final List segments = new ArrayList<>(actionClient.getPublishedSegments()); Collections.sort(segments); return Pair.of(status, segments); } else { - throw new ISE("task is not ready"); + throw new ISE("task[%s] is not ready", task.getId()); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 74b8b4b0e22d..b687c93e7f89 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1202,7 +1202,7 @@ private static List getDimensionSchema(DimensionSchema mixedTyp ); } - private static void assertIngestionSchema( + private void assertIngestionSchema( List ingestionSchemas, List expectedDimensionsSpecs, List expectedMetricsSpec, @@ -1245,7 +1245,7 @@ private static void assertIngestionSchema( ); } - private static void assertIngestionSchema( + private void assertIngestionSchema( List ingestionSchemas, List expectedDimensionsSpecs, List expectedMetricsSpec, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java index c77f063228f2..a46a5d6268df 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java @@ -27,14 +27,20 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.timeline.DataSegment; import org.apache.hadoop.yarn.util.ApplicationClassLoader; import org.easymock.EasyMock; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; import java.net.URLClassLoader; +import java.util.Collections; +import java.util.List; public class HadoopTaskTest { @@ -63,6 +69,31 @@ public boolean isReady(TaskActionClient taskActionClient) return false; } + @Override + public boolean requireLockInputSegments() + { + return true; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return true; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 97279f94b643..0a89b49ba3e1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -38,30 +38,16 @@ 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.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskReportFileWriter; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.actions.LockAcquireAction; -import org.apache.druid.indexing.common.actions.LockListAction; -import org.apache.druid.indexing.common.actions.LockTryAcquireAction; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.TaskAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; -import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -79,15 +65,10 @@ import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.loading.DataSegmentKiller; -import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager; import org.apache.druid.segment.loading.StorageLocationConfig; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; import org.apache.druid.segment.transform.ExpressionTransform; @@ -96,9 +77,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; -import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -119,7 +98,8 @@ import java.util.Map; import java.util.Set; -public class IndexTaskTest +// TODO: change segmentGranularity +public class IndexTaskTest extends IngestionTestBase { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -144,58 +124,28 @@ public class IndexTaskTest 0 ); - private DataSegmentPusher pusher; private SegmentLoader segmentLoader; - private List segments; private static final IndexSpec indexSpec = new IndexSpec(); private final ObjectMapper jsonMapper; private IndexMergerV9 indexMergerV9; private IndexIO indexIO; - private volatile int segmentAllocatePartitionCounter; - private File reportsFile; private RowIngestionMetersFactory rowIngestionMetersFactory; + private TestTaskRunner taskRunner; public IndexTaskTest() { - TestUtils testUtils = new TestUtils(); - jsonMapper = testUtils.getTestObjectMapper(); + jsonMapper = getObjectMapper(); - indexMergerV9 = testUtils.getTestIndexMergerV9(); - indexIO = testUtils.getTestIndexIO(); - rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); + indexMergerV9 = getIndexMerger(); + indexIO = getIndexIO(); + rowIngestionMetersFactory = getRowIngestionMetersFactory(); } @Before public void setup() throws IOException { - reportsFile = temporaryFolder.newFile( - StringUtils.format("IndexTaskTestReports-%s.json", System.currentTimeMillis()) - ); - - final File deepStorageDir = temporaryFolder.newFolder(); final File cacheDir = temporaryFolder.newFolder(); - - pusher = new LocalDataSegmentPusher( - new LocalDataSegmentPusherConfig() - { - @Override - public File getStorageDirectory() - { - return deepStorageDir; - } - } - ) - { - @Override - public DataSegment push(final File dataSegmentFile, final DataSegment segment, final boolean useUniquePath) - throws IOException - { - final DataSegment returnSegment = super.push(dataSegmentFile, segment, useUniquePath); - segments.add(returnSegment); - return returnSegment; - } - }; segmentLoader = new SegmentLoaderLocalCacheManager( indexIO, new SegmentLoaderConfig() @@ -210,14 +160,7 @@ public List getLocations() }, jsonMapper ); - segments = new ArrayList<>(); - - } - - @After - public void teardown() - { - reportsFile.delete(); + taskRunner = new TestTaskRunner(); } @Test @@ -237,6 +180,7 @@ public void testDeterminePartitions() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, null, @@ -283,6 +227,7 @@ public void testTransformSpec() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, new TransformSpec( @@ -330,6 +275,7 @@ public void testWithArbitraryGranularity() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, new ArbitraryGranularitySpec( @@ -366,6 +312,7 @@ public void testIntervalBucketing() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, new UniformGranularitySpec( @@ -403,6 +350,7 @@ public void testNumShardsProvided() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, null, @@ -441,6 +389,7 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, null, @@ -453,7 +402,7 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception rowIngestionMetersFactory ); - runTask(indexTask); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(2, segments.size()); @@ -502,7 +451,6 @@ public void testNumShardsAndPartitionDimensionsProvided() throws Exception @Test public void testAppendToExisting() throws Exception { - segmentAllocatePartitionCounter = 0; File tmpDir = temporaryFolder.newFolder(); File tmpFile = File.createTempFile("druid", "index", tmpDir); @@ -516,6 +464,7 @@ public void testAppendToExisting() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, null, @@ -532,7 +481,7 @@ public void testAppendToExisting() throws Exception final List segments = runTask(indexTask).rhs; - Assert.assertEquals(2, segmentAllocatePartitionCounter); + Assert.assertEquals(2, taskRunner.getTaskActionClient().getActionCount(SegmentAllocateAction.class)); Assert.assertEquals(2, segments.size()); Assert.assertEquals("test", segments.get(0).getDataSource()); @@ -562,6 +511,7 @@ public void testIntervalNotSpecified() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, new UniformGranularitySpec( @@ -614,6 +564,7 @@ public void testCSVFileWithHeader() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -666,6 +617,7 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -724,6 +676,7 @@ public void testWithSmallMaxTotalRows() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, new UniformGranularitySpec( @@ -768,6 +721,7 @@ public void testPerfectRollup() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, new UniformGranularitySpec( @@ -812,6 +766,7 @@ public void testBestEffortRollup() throws Exception null, null, createIngestionSpec( + jsonMapper, tmpDir, null, new UniformGranularitySpec( @@ -833,9 +788,9 @@ public void testBestEffortRollup() throws Exception Assert.assertEquals(5, segments.size()); + final Interval expectedInterval = Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"); for (int i = 0; i < 5; i++) { final DataSegment segment = segments.get(i); - final Interval expectedInterval = Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"); Assert.assertEquals("test", segment.getDataSource()); Assert.assertEquals(expectedInterval, segment.getInterval()); @@ -875,6 +830,7 @@ public void testIgnoreParseException() throws Exception // GranularitySpec.intervals and numShards must be null to verify reportParseException=false is respected both in // IndexTask.determineShardSpecs() and IndexTask.generateAndPublishSegments() final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -928,6 +884,7 @@ public void testReportParseException() throws Exception } final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -1016,6 +973,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception ); final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + jsonMapper, tmpDir, new JSONParseSpec( new TimestampSpec( @@ -1138,6 +1096,7 @@ public void testMultipleParseExceptionsFailure() throws Exception ); final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -1253,6 +1212,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc ); final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -1356,6 +1316,7 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception } final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -1426,6 +1387,7 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception } final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec( + jsonMapper, tmpDir, new CSVParseSpec( new TimestampSpec( @@ -1481,174 +1443,13 @@ public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus Assert.assertTrue(status.getErrorMsg().contains("Max parse exceptions exceeded, terminating task...")); } - private Pair> runTask(IndexTask indexTask) throws Exception + private Pair> runTask(IndexTask task) throws Exception { - final TaskActionClient actionClient = new TaskActionClient() - { - @Override - public RetType submit(TaskAction taskAction) - { - if (taskAction instanceof LockListAction) { - return (RetType) Collections.singletonList( - new TaskLock( - TaskLockType.EXCLUSIVE, - "", - "", - Intervals.of("2014/P1Y"), DateTimes.nowUtc().toString(), - Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY - ) - ); - } - - if (taskAction instanceof LockAcquireAction) { - return (RetType) new TaskLock( - TaskLockType.EXCLUSIVE, - "groupId", - "test", - ((LockAcquireAction) taskAction).getInterval(), - DateTimes.nowUtc().toString(), - Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY - ); - } - - if (taskAction instanceof LockTryAcquireAction) { - return (RetType) new TaskLock( - TaskLockType.EXCLUSIVE, - "groupId", - "test", - ((LockTryAcquireAction) taskAction).getInterval(), - DateTimes.nowUtc().toString(), - Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY - ); - } - - if (taskAction instanceof SegmentTransactionalInsertAction) { - return (RetType) SegmentPublishResult.ok(((SegmentTransactionalInsertAction) taskAction).getSegments()); - } - - if (taskAction instanceof SegmentAllocateAction) { - SegmentAllocateAction action = (SegmentAllocateAction) taskAction; - Interval interval = action.getPreferredSegmentGranularity().bucket(action.getTimestamp()); - ShardSpec shardSpec = new NumberedShardSpec(segmentAllocatePartitionCounter++, 0); - return (RetType) new SegmentIdWithShardSpec(action.getDataSource(), interval, "latestVersion", shardSpec); - } - - return null; - } - }; - - final DataSegmentKiller killer = new DataSegmentKiller() - { - @Override - public void kill(DataSegment segment) - { - - } - - @Override - public void killAll() - { - - } - }; - - final TaskToolbox box = new TaskToolbox( - null, - actionClient, - null, - pusher, - killer, - null, - null, - null, - null, - null, - null, - null, - null, - null, - jsonMapper, - temporaryFolder.newFolder(), - indexIO, - null, - null, - null, - indexMergerV9, - null, - null, - null, - null, - new TaskReportFileWriter(reportsFile) - ); - - indexTask.isReady(box.getTaskActionClient()); - TaskStatus status = indexTask.run(box); - - segments.sort((s1, s2) -> { - final int comp = Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval()); - if (comp != 0) { - return comp; - } - //noinspection SubtractionInCompareTo - return s1.getShardSpec().getPartitionNum() - s2.getShardSpec().getPartitionNum(); - }); - + final TaskStatus status = taskRunner.run(task).get(); + final List segments = taskRunner.getPublishedSegments(); return Pair.of(status, segments); } - private IndexTask.IndexIngestionSpec createIngestionSpec( - File baseDir, - ParseSpec parseSpec, - GranularitySpec granularitySpec, - IndexTuningConfig tuningConfig, - boolean appendToExisting - ) - { - return createIngestionSpec(baseDir, parseSpec, TransformSpec.NONE, granularitySpec, tuningConfig, appendToExisting); - } - - private IndexTask.IndexIngestionSpec createIngestionSpec( - File baseDir, - ParseSpec parseSpec, - TransformSpec transformSpec, - GranularitySpec granularitySpec, - IndexTuningConfig tuningConfig, - boolean appendToExisting - ) - { - return new IndexTask.IndexIngestionSpec( - new DataSchema( - "test", - jsonMapper.convertValue( - new StringInputRowParser( - parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC, - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec != null ? granularitySpec : new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/2015")) - ), - transformSpec, - jsonMapper - ), - new IndexTask.IndexIOConfig( - new LocalFirehoseFactory( - baseDir, - "druid*", - null - ), - appendToExisting - ), - tuningConfig - ); - } - private static IndexTuningConfig createTuningConfigWithMaxRowsPerSegment( int maxRowsPerSegment, boolean forceGuaranteedRollup @@ -1721,7 +1522,7 @@ static IndexTuningConfig createTuningConfig( private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException { Map taskReports = jsonMapper.readValue( - reportsFile, + taskRunner.getTaskReportsFile(), new TypeReference>() { } @@ -1730,4 +1531,67 @@ private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOExcep taskReports ); } + + public static IndexTask.IndexIngestionSpec createIngestionSpec( + ObjectMapper objectMapper, + File baseDir, + ParseSpec parseSpec, + GranularitySpec granularitySpec, + IndexTuningConfig tuningConfig, + boolean appendToExisting + ) + { + return createIngestionSpec( + objectMapper, + baseDir, + parseSpec, + TransformSpec.NONE, + granularitySpec, + tuningConfig, + appendToExisting + ); + } + + public static IndexTask.IndexIngestionSpec createIngestionSpec( + ObjectMapper objectMapper, + File baseDir, + ParseSpec parseSpec, + TransformSpec transformSpec, + GranularitySpec granularitySpec, + IndexTuningConfig tuningConfig, + boolean appendToExisting + ) + { + return new IndexTask.IndexIngestionSpec( + new DataSchema( + "test", + objectMapper.convertValue( + new StringInputRowParser( + parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC, + null + ), + Map.class + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }, + granularitySpec != null ? granularitySpec : new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/2015")) + ), + transformSpec, + objectMapper + ), + new IndexTask.IndexIOConfig( + new LocalFirehoseFactory( + baseDir, + "druid*", + null + ), + appendToExisting + ), + tuningConfig + ); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index c70a71a939b5..15e9aa5e12ee 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -20,44 +20,65 @@ package org.apache.druid.indexing.common.task; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; +import com.google.common.base.Optional; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskReportFileWriter; +import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClient; +import org.apache.druid.indexing.common.actions.SegmentInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionToolbox; import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; +import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorage; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; +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.metadata.EntryExistsException; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegmentManagerConfig; import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.metadata.SQLMetadataSegmentManager; import org.apache.druid.metadata.TestDerbyConnector; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; -import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.segment.loading.LocalDataSegmentPusher; +import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; +import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.timeline.DataSegment; +import org.junit.After; import org.junit.Before; import org.junit.Rule; +import org.junit.rules.TemporaryFolder; import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; -import java.util.Map; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executor; public abstract class IngestionTestBase { - public static final String DATA_SOURCE = "test"; + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); @@ -66,25 +87,41 @@ public abstract class IngestionTestBase private final ObjectMapper objectMapper = testUtils.getTestObjectMapper(); private TaskStorage taskStorage; private IndexerSQLMetadataStorageCoordinator storageCoordinator; + private MetadataSegmentManager segmentManager; private TaskLockbox lockbox; @Before - public void setUp() + public void setUp() throws IOException { + temporaryFolder.create(); + final SQLMetadataConnector connector = derbyConnectorRule.getConnector(); connector.createTaskTables(); + connector.createSegmentTable(); taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); storageCoordinator = new IndexerSQLMetadataStorageCoordinator( objectMapper, derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnectorRule.getConnector() ); - lockbox = new TaskLockbox(taskStorage); + segmentManager = new SQLMetadataSegmentManager( + objectMapper, + MetadataSegmentManagerConfig::new, + derbyConnectorRule.metadataTablesConfigSupplier(), + derbyConnectorRule.getConnector() + ); + lockbox = new TaskLockbox(taskStorage, storageCoordinator); } - public LocalTaskActionClient createActionClient(Task task) + @After + public void tearDown() { - return new LocalTaskActionClient(task, taskStorage, createTaskActionToolbox(), new TaskAuditLogConfig(false)); + temporaryFolder.delete(); + } + + public TestLocalTaskActionClient createActionClient(Task task) + { + return new TestLocalTaskActionClient(task); } public void prepareTaskForLocking(Task task) throws EntryExistsException @@ -108,6 +145,16 @@ public TaskStorage getTaskStorage() return taskStorage; } + public IndexerMetadataStorageCoordinator getMetadataStorageCoordinator() + { + return storageCoordinator; + } + + public MetadataSegmentManager getMetadataSegmentManager() + { + return segmentManager; + } + public TaskLockbox getLockbox() { return lockbox; @@ -118,6 +165,11 @@ public IndexerSQLMetadataStorageCoordinator getStorageCoordinator() return storageCoordinator; } + public RowIngestionMetersFactory getRowIngestionMetersFactory() + { + return testUtils.getRowIngestionMetersFactory(); + } + public TaskActionToolbox createTaskActionToolbox() { storageCoordinator.start(); @@ -140,53 +192,167 @@ public IndexMergerV9 getIndexMerger() return testUtils.getTestIndexMergerV9(); } - public IndexTask.IndexIngestionSpec createIngestionSpec( - File baseDir, - ParseSpec parseSpec, - GranularitySpec granularitySpec, - IndexTuningConfig tuningConfig, - boolean appendToExisting - ) - { - return createIngestionSpec(baseDir, parseSpec, TransformSpec.NONE, granularitySpec, tuningConfig, appendToExisting); - } - - public IndexTask.IndexIngestionSpec createIngestionSpec( - File baseDir, - ParseSpec parseSpec, - TransformSpec transformSpec, - GranularitySpec granularitySpec, - IndexTuningConfig tuningConfig, - boolean appendToExisting - ) - { - return new IndexTask.IndexIngestionSpec( - new DataSchema( - DATA_SOURCE, - objectMapper.convertValue( - new StringInputRowParser(parseSpec, null), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec != null ? granularitySpec : new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/2015")) - ), - transformSpec, - objectMapper - ), - new IndexTask.IndexIOConfig( - new LocalFirehoseFactory( - baseDir, - "druid*", - null - ), - appendToExisting - ), - tuningConfig - ); + public class TestLocalTaskActionClient extends LocalTaskActionClient + { + private final Set publishedSegments = new HashSet<>(); + + private TestLocalTaskActionClient(Task task) + { + super(task, taskStorage, createTaskActionToolbox(), new TaskAuditLogConfig(false)); + } + + @Override + public RetType submit(TaskAction taskAction) + { + final RetType result = super.submit(taskAction); + if (taskAction instanceof SegmentTransactionalInsertAction) { + publishedSegments.addAll(((SegmentTransactionalInsertAction) taskAction).getSegments()); + } else if (taskAction instanceof SegmentInsertAction) { + publishedSegments.addAll(((SegmentInsertAction) taskAction).getSegments()); + } + return result; + } + + public Set getPublishedSegments() + { + return publishedSegments; + } + } + + public class TestTaskRunner implements TaskRunner + { + private TestLocalTaskActionClient taskActionClient; + private File taskReportsFile; + + @Override + public List>> restore() + { + throw new UnsupportedOperationException(); + } + + @Override + public void start() + { + throw new UnsupportedOperationException(); + } + + @Override + public void registerListener(TaskRunnerListener listener, Executor executor) + { + throw new UnsupportedOperationException(); + } + + @Override + public void unregisterListener(String listenerId) + { + throw new UnsupportedOperationException(); + } + + public TestLocalTaskActionClient getTaskActionClient() + { + return taskActionClient; + } + + public File getTaskReportsFile() + { + return taskReportsFile; + } + + public List getPublishedSegments() + { + final List segments = new ArrayList<>(taskActionClient.getPublishedSegments()); + Collections.sort(segments); + return segments; + } + + @Override + public ListenableFuture run(Task task) + { + try { + lockbox.add(task); + taskStorage.insert(task, TaskStatus.running(task.getId())); + taskActionClient = createActionClient(task); + taskReportsFile = temporaryFolder.newFile( + StringUtils.format("ingestionTestBase-%s.json", System.currentTimeMillis()) + ); + + final TaskToolbox box = new TaskToolbox( + null, + taskActionClient, + null, + new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig()), + new NoopDataSegmentKiller(), + null, + null, + null, + null, + null, + null, + null, + null, + null, + objectMapper, + temporaryFolder.newFolder(), + getIndexIO(), + null, + null, + null, + getIndexMerger(), + null, + null, + null, + null, + new TaskReportFileWriter(taskReportsFile) + ); + + if (task.isReady(box.getTaskActionClient())) { + return Futures.immediateFuture(task.run(box)); + } else { + throw new ISE("task is not ready"); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + lockbox.remove(task); + } + } + + @Override + public void shutdown(String taskid, String reason) + { + throw new UnsupportedOperationException(); + } + + @Override + public void stop() + { + throw new UnsupportedOperationException(); + } + + @Override + public Collection getRunningTasks() + { + throw new UnsupportedOperationException(); + } + + @Override + public Collection getPendingTasks() + { + throw new UnsupportedOperationException(); + } + + @Override + public Collection getKnownTasks() + { + throw new UnsupportedOperationException(); + } + + @Override + public Optional getScalingStats() + { + throw new UnsupportedOperationException(); + } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java new file mode 100644 index 000000000000..33bf5b35d3d7 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.Set; + +public class KillTaskTest extends IngestionTestBase +{ + private static final String DATA_SOURCE = "dataSource"; + + private TaskRunner taskRunner; + + @Before + public void setup() + { + taskRunner = new TestTaskRunner(); + } + + @Test + public void testKill() throws Exception + { + final Set segments = ImmutableSet.of( + newSegment(Intervals.of("2019-01-01/2019-02-01")), + newSegment(Intervals.of("2019-02-01/2019-03-01")), + newSegment(Intervals.of("2019-03-01/2019-04-01")), + newSegment(Intervals.of("2019-04-01/2019-05-01")) + ); + final Set announced = getMetadataStorageCoordinator().announceHistoricalSegments(segments); + + Assert.assertEquals(segments, announced); + + Assert.assertTrue( + getMetadataSegmentManager().removeSegment( + DATA_SOURCE, + newSegment(Intervals.of("2019-02-01/2019-03-01")).getId().toString() + ) + ); + Assert.assertTrue( + getMetadataSegmentManager().removeSegment( + DATA_SOURCE, + newSegment(Intervals.of("2019-03-01/2019-04-01")).getId().toString() + ) + ); + + final KillTask task = new KillTask(null, DATA_SOURCE, Intervals.of("2019-03-01/2019-04-01"), null); + + Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode()); + + final List unusedSegments = getMetadataStorageCoordinator().getUnusedSegmentsForInterval( + DATA_SOURCE, + Intervals.of("2019/2020") + ); + + Assert.assertEquals(ImmutableList.of(newSegment(Intervals.of("2019-02-01/2019-03-01"))), unusedSegments); + Assert.assertEquals( + ImmutableList.of( + newSegment(Intervals.of("2019-01-01/2019-02-01")), + newSegment(Intervals.of("2019-04-01/2019-05-01")) + ), + getMetadataStorageCoordinator().getUsedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020")) + ); + } + + private static DataSegment newSegment(Interval interval) + { + return new DataSegment( + DATA_SOURCE, + interval, + "version", + null, + null, + null, + null, + 9, + 10L + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 348c349184f3..3bb1fc0ae2cc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -968,7 +968,7 @@ private TaskToolbox makeToolbox( ) { final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, true, null, null); - final TaskLockbox taskLockbox = new TaskLockbox(taskStorage); + final TaskLockbox taskLockbox = new TaskLockbox(taskStorage, mdc); try { taskStorage.insert(task, TaskStatus.running(task.getId())); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 8792956bc95a..e1ba95871d23 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -49,16 +49,14 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.security.AllowAllAuthorizer; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; -import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Rule; import org.junit.rules.TemporaryFolder; @@ -230,18 +228,7 @@ public File getStorageDirectory() } } ), - new DataSegmentKiller() - { - @Override - public void kill(DataSegment segment) - { - } - - @Override - public void killAll() - { - } - }, + new NoopDataSegmentKiller(), null, null, null, @@ -368,12 +355,6 @@ static class LocalParallelIndexTaskClient extends ParallelIndexTaskClient this.supervisorTask = supervisorTask; } - @Override - public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException - { - return supervisorTask.allocateNewSegment(timestamp); - } - @Override public void report(String supervisorTaskId, List pushedSegments) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 704c5fa4500b..db7418774a13 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -24,17 +24,14 @@ import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.LockListAction; -import org.apache.druid.indexing.common.actions.SurrogateAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; @@ -48,10 +45,11 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.After; @@ -543,7 +541,6 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), supervisorTask, - this, new ParallelIndexIngestionSpec( getIngestionSchema().getDataSchema(), new ParallelIndexIOConfig( @@ -568,7 +565,6 @@ private class TestSubTaskSpec extends ParallelIndexSubTaskSpec String id, String groupId, ParallelIndexSupervisorTask supervisorTask, - SinglePhaseParallelIndexTaskRunner runner, ParallelIndexIngestionSpec ingestionSpec, Map context, InputSplit inputSplit @@ -624,6 +620,7 @@ public ParallelIndexSubTask newSubTask(int numAttempts) private class TestSubTask extends ParallelIndexSubTask { + private final IndexTaskClientFactory taskClientFactory; private volatile TaskState state = TaskState.RUNNING; TestSubTask( @@ -646,12 +643,7 @@ private class TestSubTask extends ParallelIndexSubTask null, taskClientFactory ); - } - - @Override - public boolean isReady(TaskActionClient taskActionClient) - { - return true; + this.taskClientFactory = taskClientFactory; } @Override @@ -661,30 +653,32 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception Thread.sleep(100); } - final TestFirehose firehose = (TestFirehose) getIngestionSchema().getIOConfig().getFirehoseFactory(); + // build LocalParallelIndexTaskClient + final ParallelIndexTaskClient taskClient = taskClientFactory.build(null, getId(), 0, null, 0); - final List locks = toolbox.getTaskActionClient() - .submit(new SurrogateAction<>(getSupervisorTaskId(), new LockListAction())); - Preconditions.checkState(locks.size() == 1, "There should be a single lock"); + final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); - task.getRunner().collectReport( - new PushedSegmentsReport( - getId(), - Collections.singletonList( - new DataSegment( - getDataSource(), - Intervals.of("2017/2018"), - locks.get(0).getVersion(), - null, - null, - null, - new NumberedShardSpec(firehose.ids.get(0), NUM_SUB_TASKS), - 0, - 1L - ) - ) - ) + final SegmentIdWithShardSpec segmentIdentifier = segmentAllocator.allocate( + new MapBasedInputRow(DateTimes.of("2017-01-01"), Collections.emptyList(), Collections.emptyMap()), + getId(), + null, + true ); + + final DataSegment segment = new DataSegment( + segmentIdentifier.getDataSource(), + segmentIdentifier.getInterval(), + segmentIdentifier.getVersion(), + null, + null, + null, + segmentIdentifier.getShardSpec(), + 0, + 1L + ); + // TODO: maybe check shardSpec?? + + taskClient.report(getId(), Collections.singletonList(segment)); return TaskStatus.fromCode(getId(), state); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index cfd02ac18cee..e82f479a7599 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -48,12 +48,15 @@ import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +// TODO: change segmentGranularity public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest { private File inputDir; @@ -152,19 +155,33 @@ public void testWithoutInterval() throws Exception runTestWithoutIntervalTask(); // Read the segments for one day. - final Interval interval = Intervals.of("2017-12-24/P1D"); - final List oldSegments = - getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); - Assert.assertEquals(1, oldSegments.size()); + final Interval interval = Intervals.of("2017-12-24/P5D"); + final List allOldSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + final Map> oldIntervalToSegments = new HashMap<>(); + allOldSegments.forEach( + segment -> oldIntervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); // Reingest the same data. Each segment should get replaced by a segment with a newer version. runTestWithoutIntervalTask(); // Verify that the segment has been replaced. - final List newSegments = - getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); - Assert.assertEquals(1, newSegments.size()); - Assert.assertTrue(oldSegments.get(0).getVersion().compareTo(newSegments.get(0).getVersion()) < 0); + final List allNewSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + final Map> newIntervalToSegments = new HashMap<>(); + allNewSegments.forEach( + segment -> newIntervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + Assert.assertEquals(allOldSegments.size(), allNewSegments.size()); + for (Entry> entry : oldIntervalToSegments.entrySet()) { + final List oldSegments = entry.getValue(); + final List newSegments = newIntervalToSegments.get(entry.getKey()); + Assert.assertEquals(oldSegments.size(), newSegments.size()); + newSegments.forEach( + newSegment -> Assert.assertTrue( + oldSegments.stream().allMatch(oldSegment -> oldSegment.getMinorVersion() < newSegment.getMinorVersion()) + ) + ); + } } @Test() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 70e5544b00eb..a776a159cf2c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -59,6 +59,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; @@ -97,8 +98,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +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.IntStream; @@ -112,6 +115,7 @@ public class IngestSegmentFirehoseFactoryTest private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; private static final TaskStorage TASK_STORAGE; + private static final IndexerSQLMetadataStorageCoordinator MDC; private static final TaskLockbox TASK_LOCKBOX; private static final Task TASK; @@ -125,7 +129,48 @@ public class IngestSegmentFirehoseFactoryTest { } ); - TASK_LOCKBOX = new TaskLockbox(TASK_STORAGE); + MDC = new IndexerSQLMetadataStorageCoordinator(null, null, null) + { + private final Set published = new HashSet<>(); + + @Override + public List getUsedSegmentsForInterval(String dataSource, Interval interval) + { + return ImmutableList.copyOf(segmentSet); + } + + @Override + public List getUsedSegmentsForIntervals(String dataSource, List interval) + { + return ImmutableList.copyOf(segmentSet); + } + + @Override + public List getUnusedSegmentsForInterval(String dataSource, Interval interval) + { + return ImmutableList.of(); + } + + @Override + public Set announceHistoricalSegments(Set segments) + { + Set added = new HashSet<>(); + for (final DataSegment segment : segments) { + if (published.add(segment)) { + added.add(segment); + } + } + + return ImmutableSet.copyOf(added); + } + + @Override + public void deleteSegments(Set segments) + { + // do nothing + } + }; + TASK_LOCKBOX = new TaskLockbox(TASK_STORAGE, MDC); TASK = NoopTask.create(); TASK_LOCKBOX.add(TASK); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java index 24291e8257a0..6a23f43134b1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java @@ -25,18 +25,21 @@ import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.LockAcquireAction; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.LockReleaseAction; import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.Assert; +import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; /** @@ -60,6 +63,31 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { @@ -70,7 +98,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Acquire lock for first interval final TaskLock lock1 = toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, interval1, 5000) + new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, interval1, 5000) ); Assert.assertNotNull(lock1); final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); @@ -81,7 +109,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Acquire lock for second interval final TaskLock lock2 = toolbox.getTaskActionClient().submit( - new LockAcquireAction(TaskLockType.EXCLUSIVE, interval2, 5000) + new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, interval2, 5000) ); Assert.assertNotNull(lock2); final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 49315d3973d0..c17b09d4883f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.loading.NoopDataSegmentArchiver; import org.apache.druid.segment.loading.NoopDataSegmentKiller; @@ -41,7 +42,9 @@ import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; +import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -49,9 +52,11 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -181,6 +186,31 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 6f2b99218a26..374b338accc1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -46,15 +46,18 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.SegmentInsertAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -80,6 +83,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.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -140,6 +144,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -527,7 +532,7 @@ private TaskToolboxFactory setUpTaskToolboxFactory( Preconditions.checkNotNull(taskStorage); Preconditions.checkNotNull(emitter); - taskLockbox = new TaskLockbox(taskStorage); + taskLockbox = new TaskLockbox(taskStorage, mdc); tac = new LocalTaskActionClientFactory( taskStorage, new TaskActionToolbox( @@ -931,18 +936,49 @@ public String getType() return "test"; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction()) + final Interval interval = Intervals.of("2012-01-01/P1D"); + final TimeChunkLockTryAcquireAction action = new TimeChunkLockTryAcquireAction( + TaskLockType.EXCLUSIVE, + interval ); + final TaskLock lock = toolbox.getTaskActionClient().submit(action); + if (lock == null) { + throw new ISE("Failed to get a lock"); + } + final DataSegment segment = DataSegment.builder() .dataSource("ds") - .interval(Intervals.of("2012-01-01/P1D")) - .version(myLock.getVersion()) + .interval(interval) + .version(lock.getVersion()) .build(); toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); @@ -968,6 +1004,32 @@ public String getType() return "test"; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { @@ -1002,6 +1064,31 @@ public String getType() return "test"; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 10b46c88fba6..d535aae6e6d5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLockType; @@ -31,6 +32,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; import org.joda.time.Interval; import org.junit.After; @@ -72,7 +74,10 @@ public void setup() ) ); - lockbox = new TaskLockbox(taskStorage); + lockbox = new TaskLockbox( + taskStorage, + new IndexerSQLMetadataStorageCoordinator(objectMapper, derby.metadataTablesConfigSupplier().get(), derbyConnector) + ); service = Executors.newFixedThreadPool(2); } @@ -82,6 +87,17 @@ public void teardown() service.shutdownNow(); } + private LockResult tryTimeChunkLock(TaskLockType lockType, Task task, Interval interval) + { + return lockbox.tryLock(task, new TimeChunkLockRequest(lockType, task, interval, null)); + } + + private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval) + throws InterruptedException + { + return lockbox.lock(task, new TimeChunkLockRequest(lockType, task, interval, null)); + } + @Test(timeout = 60_000L) public void testDoInCriticalSectionWithDifferentTasks() throws ExecutionException, InterruptedException, EntryExistsException @@ -99,13 +115,13 @@ public void testDoInCriticalSectionWithDifferentTasks() // lowPriorityTask acquires a lock first and increases the int of intSupplier in the critical section final Future lowPriorityFuture = service.submit(() -> { - final LockResult result = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); + final LockResult result = tryTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); Assert.assertTrue(result.isOk()); Assert.assertFalse(result.isRevoked()); return lockbox.doInCriticalSection( lowPriorityTask, - Collections.singletonList(interval), + Collections.singletonMap(interval, Collections.emptyList()), CriticalAction.builder() .onValidLocks( () -> { @@ -129,13 +145,13 @@ public void testDoInCriticalSectionWithDifferentTasks() // section final Future highPriorityFuture = service.submit(() -> { latch.await(); - final LockResult result = lockbox.lock(TaskLockType.EXCLUSIVE, highPriorityTask, interval); + final LockResult result = acquireTimeChunkLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval); Assert.assertTrue(result.isOk()); Assert.assertFalse(result.isRevoked()); return lockbox.doInCriticalSection( highPriorityTask, - Collections.singletonList(interval), + Collections.singletonMap(interval, Collections.emptyList()), CriticalAction.builder() .onValidLocks( () -> { @@ -158,7 +174,7 @@ public void testDoInCriticalSectionWithDifferentTasks() Assert.assertEquals(2, highPriorityFuture.get().intValue()); // the lock for lowPriorityTask must be revoked by the highPriorityTask after its work is done in critical section - final LockResult result = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); + final LockResult result = tryTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); Assert.assertFalse(result.isOk()); Assert.assertTrue(result.isRevoked()); } @@ -176,7 +192,7 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception taskStorage.insert(task, TaskStatus.running(task.getId())); for (Interval interval : intervals) { - final LockResult result = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval); + final LockResult result = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval); Assert.assertTrue(result.isOk()); } @@ -185,7 +201,7 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception final Future future1 = service.submit(() -> lockbox.doInCriticalSection( task, - ImmutableList.of(intervals.get(0), intervals.get(1)), + ImmutableMap.of(intervals.get(0), Collections.emptyList(), intervals.get(1), Collections.emptyList()), CriticalAction.builder() .onValidLocks( () -> { @@ -208,7 +224,7 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception latch.await(); return lockbox.doInCriticalSection( task, - ImmutableList.of(intervals.get(1), intervals.get(2)), + ImmutableMap.of(intervals.get(1), Collections.emptyList(), intervals.get(2), Collections.emptyList()), CriticalAction.builder() .onValidLocks( () -> { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index a2a14a006bba..a86fc1f69361 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -27,9 +27,11 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.Iterables; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.AbstractTask; @@ -40,11 +42,22 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.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.NumberedShardSpecFactory; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; @@ -53,6 +66,7 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -70,8 +84,9 @@ public class TaskLockboxTest @Rule public ExpectedException expectedException = ExpectedException.none(); - private final ObjectMapper objectMapper = new DefaultObjectMapper(); + private ObjectMapper objectMapper; private TaskStorage taskStorage; + private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private TaskLockbox lockbox; @Rule @@ -80,14 +95,20 @@ public class TaskLockboxTest @Before public void setup() { + objectMapper = TestHelper.makeJsonMapper(); + objectMapper.registerSubtypes(NumberedShardSpec.class, HashBasedNumberedShardSpec.class); + final TestDerbyConnector derbyConnector = derby.getConnector(); derbyConnector.createTaskTables(); + derbyConnector.createPendingSegmentsTable(); + derbyConnector.createSegmentTable(); + final MetadataStorageTablesConfig tablesConfig = derby.metadataTablesConfigSupplier().get(); taskStorage = new MetadataTaskStorage( derbyConnector, new TaskStorageConfig(null), new DerbyMetadataStorageActionHandlerFactory( derbyConnector, - derby.metadataTablesConfigSupplier().get(), + tablesConfig, objectMapper ) ); @@ -95,7 +116,26 @@ public void setup() EmittingLogger.registerEmitter(emitter); EasyMock.replay(emitter); - lockbox = new TaskLockbox(taskStorage); + metadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator(objectMapper, tablesConfig, derbyConnector); + + lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + } + + private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval, long timeoutMs) + throws InterruptedException + { + return lockbox.lock(task, new TimeChunkLockRequest(lockType, task, interval, null), timeoutMs); + } + + private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interval interval) + throws InterruptedException + { + return lockbox.lock(task, new TimeChunkLockRequest(lockType, task, interval, null)); + } + + private LockResult tryTimeChunkLock(TaskLockType lockType, Task task, Interval interval) + { + return lockbox.tryLock(task, new TimeChunkLockRequest(lockType, task, interval, null)); } @Test @@ -103,13 +143,13 @@ public void testLock() throws InterruptedException { Task task = NoopTask.create(); lockbox.add(task); - Assert.assertNotNull(lockbox.lock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02"))); + Assert.assertNotNull(acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02"))); } @Test(expected = IllegalStateException.class) public void testLockForInactiveTask() throws InterruptedException { - lockbox.lock(TaskLockType.EXCLUSIVE, NoopTask.create(), Intervals.of("2015-01-01/2015-01-02")); + acquireTimeChunkLock(TaskLockType.EXCLUSIVE, NoopTask.create(), Intervals.of("2015-01-01/2015-01-02")); } @Test @@ -120,7 +160,7 @@ public void testLockAfterTaskComplete() throws InterruptedException exception.expectMessage("Unable to grant lock to inactive Task"); lockbox.add(task); lockbox.remove(task); - lockbox.lock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02")); + acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02")); } @Test @@ -135,7 +175,7 @@ public void testTrySharedLock() final Task task = NoopTask.create(Math.min(0, (i - 1) * 10)); // the first two tasks have the same priority tasks.add(task); lockbox.add(task); - final TaskLock lock = lockbox.tryLock(TaskLockType.SHARED, task, interval).getTaskLock(); + final TaskLock lock = tryTimeChunkLock(TaskLockType.SHARED, task, interval).getTaskLock(); Assert.assertNotNull(lock); actualLocks.add(lock); } @@ -160,15 +200,15 @@ public void testTryMixedLocks() throws EntryExistsException lockbox.add(lowPriorityTask); lockbox.add(lowPriorityTask2); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval1).isOk()); - Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, lowPriorityTask, interval2).isOk()); - Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, lowPriorityTask2, interval2).isOk()); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval3).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval1).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.SHARED, lowPriorityTask, interval2).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.SHARED, lowPriorityTask2, interval2).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval3).isOk()); lockbox.add(highPiorityTask); - Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, highPiorityTask, interval1).isOk()); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval2).isOk()); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval3).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.SHARED, highPiorityTask, interval1).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval2).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval3).isOk()); Assert.assertTrue(lockbox.findLocksForTask(lowPriorityTask).stream().allMatch(TaskLock::isRevoked)); Assert.assertTrue(lockbox.findLocksForTask(lowPriorityTask2).stream().allMatch(TaskLock::isRevoked)); @@ -178,14 +218,14 @@ public void testTryMixedLocks() throws EntryExistsException lockbox.remove(highPiorityTask); lockbox.add(highPiorityTask); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval1).isOk()); - Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, highPiorityTask, interval2).isOk()); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval3).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval1).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.SHARED, highPiorityTask, interval2).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, highPiorityTask, interval3).isOk()); lockbox.add(lowPriorityTask); - Assert.assertFalse(lockbox.tryLock(TaskLockType.SHARED, lowPriorityTask, interval1).isOk()); - Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval2).isOk()); - Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval3).isOk()); + Assert.assertFalse(tryTimeChunkLock(TaskLockType.SHARED, lowPriorityTask, interval1).isOk()); + Assert.assertFalse(tryTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval2).isOk()); + Assert.assertFalse(tryTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval3).isOk()); } @Test @@ -193,24 +233,24 @@ public void testTryExclusiveLock() { Task task = NoopTask.create(); lockbox.add(task); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-03")).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-03")).isOk()); // try to take lock for task 2 for overlapping interval Task task2 = NoopTask.create(); lockbox.add(task2); - Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, task2, Intervals.of("2015-01-01/2015-01-02")).isOk()); + Assert.assertFalse(tryTimeChunkLock(TaskLockType.EXCLUSIVE, task2, Intervals.of("2015-01-01/2015-01-02")).isOk()); // task 1 unlocks the lock lockbox.remove(task); // Now task2 should be able to get the lock - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, task2, Intervals.of("2015-01-01/2015-01-02")).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, task2, Intervals.of("2015-01-01/2015-01-02")).isOk()); } @Test(expected = IllegalStateException.class) public void testTryLockForInactiveTask() { - Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, NoopTask.create(), Intervals.of("2015-01-01/2015-01-02")).isOk()); + Assert.assertFalse(tryTimeChunkLock(TaskLockType.EXCLUSIVE, NoopTask.create(), Intervals.of("2015-01-01/2015-01-02")).isOk()); } @Test @@ -221,7 +261,7 @@ public void testTryLockAfterTaskComplete() exception.expectMessage("Unable to grant lock to inactive Task"); lockbox.add(task); lockbox.remove(task); - Assert.assertFalse(lockbox.tryLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02")).isOk()); + Assert.assertFalse(tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-02")).isOk()); } @Test @@ -232,23 +272,27 @@ public void testTimeoutForLock() throws InterruptedException lockbox.add(task1); lockbox.add(task2); - Assert.assertTrue(lockbox.lock(TaskLockType.EXCLUSIVE, task1, Intervals.of("2015-01-01/2015-01-02"), 5000).isOk()); - Assert.assertFalse(lockbox.lock(TaskLockType.EXCLUSIVE, task2, Intervals.of("2015-01-01/2015-01-15"), 1000).isOk()); + Assert.assertTrue(acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task1, Intervals.of("2015-01-01/2015-01-02"), 5000).isOk()); + Assert.assertFalse(acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task2, Intervals.of("2015-01-01/2015-01-15"), 1000).isOk()); } @Test public void testSyncFromStorage() throws EntryExistsException { - final TaskLockbox originalBox = new TaskLockbox(taskStorage); + final TaskLockbox originalBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); for (int i = 0; i < 5; i++) { final Task task = NoopTask.create(); taskStorage.insert(task, TaskStatus.running(task.getId())); originalBox.add(task); Assert.assertTrue( originalBox.tryLock( - TaskLockType.EXCLUSIVE, task, - Intervals.of(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) + new TimeChunkLockRequest( + TaskLockType.EXCLUSIVE, + task, + Intervals.of(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))), + null + ) ).isOk() ); } @@ -257,7 +301,7 @@ public void testSyncFromStorage() throws EntryExistsException .flatMap(task -> taskStorage.getLocks(task.getId()).stream()) .collect(Collectors.toList()); - final TaskLockbox newBox = new TaskLockbox(taskStorage); + final TaskLockbox newBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); newBox.syncFromStorage(); Assert.assertEquals(originalBox.getAllLocks(), newBox.getAllLocks()); @@ -277,14 +321,14 @@ public void testSyncFromStorageWithMissingTaskLockPriority() throws EntryExistsE taskStorage.insert(task, TaskStatus.running(task.getId())); taskStorage.addLock( task.getId(), - new TaskLockWithoutPriority(task.getGroupId(), task.getDataSource(), Intervals.of("2017/2018"), "v1") + new IntervalLockWithoutPriority(task.getGroupId(), task.getDataSource(), Intervals.of("2017/2018"), "v1") ); final List beforeLocksInStorage = taskStorage.getActiveTasks().stream() .flatMap(t -> taskStorage.getLocks(t.getId()).stream()) .collect(Collectors.toList()); - final TaskLockbox lockbox = new TaskLockbox(taskStorage); + final TaskLockbox lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); lockbox.syncFromStorage(); final List afterLocksInStorage = taskStorage.getActiveTasks().stream() @@ -301,7 +345,7 @@ public void testSyncFromStorageWithMissingTaskPriority() throws EntryExistsExcep taskStorage.insert(task, TaskStatus.running(task.getId())); taskStorage.addLock( task.getId(), - new TaskLock( + new TimeChunkLock( TaskLockType.EXCLUSIVE, task.getGroupId(), task.getDataSource(), @@ -315,7 +359,7 @@ public void testSyncFromStorageWithMissingTaskPriority() throws EntryExistsExcep .flatMap(t -> taskStorage.getLocks(t.getId()).stream()) .collect(Collectors.toList()); - final TaskLockbox lockbox = new TaskLockbox(taskStorage); + final TaskLockbox lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); lockbox.syncFromStorage(); final List afterLocksInStorage = taskStorage.getActiveTasks().stream() @@ -332,7 +376,7 @@ public void testSyncFromStorageWithInvalidPriority() throws EntryExistsException taskStorage.insert(task, TaskStatus.running(task.getId())); taskStorage.addLock( task.getId(), - new TaskLock( + new TimeChunkLock( TaskLockType.EXCLUSIVE, task.getGroupId(), task.getDataSource(), @@ -342,7 +386,7 @@ public void testSyncFromStorageWithInvalidPriority() throws EntryExistsException ) ); - final TaskLockbox lockbox = new TaskLockbox(taskStorage); + final TaskLockbox lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("lock priority[10] is different from task priority[50]"); lockbox.syncFromStorage(); @@ -364,9 +408,14 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() throws Exception loadedMapper ) ); + IndexerMetadataStorageCoordinator loadedMetadataStorageCoordinator = new IndexerSQLMetadataStorageCoordinator( + loadedMapper, + derby.metadataTablesConfigSupplier().get(), + derbyConnector + ); - TaskLockbox theBox = new TaskLockbox(taskStorage); - TaskLockbox loadedBox = new TaskLockbox(loadedTaskStorage); + TaskLockbox theBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + TaskLockbox loadedBox = new TaskLockbox(loadedTaskStorage, loadedMetadataStorageCoordinator); Task aTask = NoopTask.create(); taskStorage.insert(aTask, TaskStatus.running(aTask.getId())); @@ -391,18 +440,18 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() throws Exception @Test public void testRevokedLockSyncFromStorage() throws EntryExistsException { - final TaskLockbox originalBox = new TaskLockbox(taskStorage); + final TaskLockbox originalBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); final Task task1 = NoopTask.create("task1", 10); taskStorage.insert(task1, TaskStatus.running(task1.getId())); originalBox.add(task1); - Assert.assertTrue(originalBox.tryLock(TaskLockType.EXCLUSIVE, task1, Intervals.of("2017/2018")).isOk()); + Assert.assertTrue(originalBox.tryLock(task1, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task1, Intervals.of("2017/2018"), null)).isOk()); // task2 revokes task1 final Task task2 = NoopTask.create("task2", 100); taskStorage.insert(task2, TaskStatus.running(task2.getId())); originalBox.add(task2); - Assert.assertTrue(originalBox.tryLock(TaskLockType.EXCLUSIVE, task2, Intervals.of("2017/2018")).isOk()); + Assert.assertTrue(originalBox.tryLock(task2, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task2, Intervals.of("2017/2018"), null)).isOk()); final Map> beforeLocksInStorage = taskStorage .getActiveTasks() @@ -417,7 +466,7 @@ public void testRevokedLockSyncFromStorage() throws EntryExistsException Assert.assertEquals(1, task2Locks.size()); Assert.assertTrue(task2Locks.get(0).isRevoked()); - final TaskLockbox newBox = new TaskLockbox(taskStorage); + final TaskLockbox newBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); newBox.syncFromStorage(); final Set afterLocksInStorage = taskStorage.getActiveTasks().stream() @@ -436,12 +485,12 @@ public void testDoInCriticalSectionWithSharedLock() throws Exception final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task task = NoopTask.create(); lockbox.add(task); - Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.SHARED, task, interval).isOk()); Assert.assertFalse( lockbox.doInCriticalSection( task, - Collections.singletonList(interval), + Collections.singletonMap(interval, Collections.emptyList()), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -453,13 +502,13 @@ public void testDoInCriticalSectionWithExclusiveLock() throws Exception final Interval interval = Intervals.of("2017-01-01/2017-01-02"); final Task task = NoopTask.create(); lockbox.add(task); - final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); Assert.assertNotNull(lock); Assert.assertTrue( lockbox.doInCriticalSection( task, - Collections.singletonList(interval), + Collections.singletonMap(interval, Collections.emptyList()), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -472,13 +521,13 @@ public void testDoInCriticalSectionWithSmallerInterval() throws Exception final Interval smallInterval = Intervals.of("2017-01-10/2017-01-11"); final Task task = NoopTask.create(); lockbox.add(task); - final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); + final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); Assert.assertNotNull(lock); Assert.assertTrue( lockbox.doInCriticalSection( task, - Collections.singletonList(smallInterval), + Collections.singletonMap(interval, Collections.emptyList()), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -492,19 +541,19 @@ public void testPreemptionAndDoInCriticalSection() throws Exception final Task task = NoopTask.create(); lockbox.add(task); taskStorage.insert(task, TaskStatus.running(task.getId())); - Assert.assertTrue(lockbox.tryLock(TaskLockType.SHARED, task, interval).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.SHARED, task, interval).isOk()); } final Task highPriorityTask = NoopTask.create(100); lockbox.add(highPriorityTask); taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); - final TaskLock lock = lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).getTaskLock(); + final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).getTaskLock(); Assert.assertNotNull(lock); Assert.assertTrue( lockbox.doInCriticalSection( highPriorityTask, - Collections.singletonList(interval), + Collections.singletonMap(interval, Collections.emptyList()), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -521,15 +570,15 @@ public void testDoInCriticalSectionWithRevokedLock() throws Exception taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); - final TaskLock lowPriorityLock = lockbox.tryLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); + final TaskLock lowPriorityLock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); Assert.assertNotNull(lowPriorityLock); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); Assert.assertFalse( lockbox.doInCriticalSection( lowPriorityTask, - Collections.singletonList(interval), + Collections.singletonMap(interval, Collections.emptyList()), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -546,15 +595,15 @@ public void testAcquireLockAfterRevoked() throws EntryExistsException, Interrupt taskStorage.insert(lowPriorityTask, TaskStatus.running(lowPriorityTask.getId())); taskStorage.insert(highPriorityTask, TaskStatus.running(highPriorityTask.getId())); - final TaskLock lowPriorityLock = lockbox.lock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); + final TaskLock lowPriorityLock = acquireTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval).getTaskLock(); Assert.assertNotNull(lowPriorityLock); - Assert.assertTrue(lockbox.tryLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); + Assert.assertTrue(tryTimeChunkLock(TaskLockType.EXCLUSIVE, highPriorityTask, interval).isOk()); Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); lockbox.unlock(highPriorityTask, interval); // Acquire again - final LockResult lockResult = lockbox.lock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); + final LockResult lockResult = acquireTimeChunkLock(TaskLockType.EXCLUSIVE, lowPriorityTask, interval); Assert.assertFalse(lockResult.isOk()); Assert.assertTrue(lockResult.isRevoked()); Assert.assertTrue(Iterables.getOnlyElement(lockbox.findLocksForTask(lowPriorityTask)).isRevoked()); @@ -572,7 +621,7 @@ public void testUnlock() throws EntryExistsException taskStorage.insert(task, TaskStatus.running(task.getId())); lockbox.add(task); Assert.assertTrue( - lockbox.tryLock( + tryTimeChunkLock( TaskLockType.EXCLUSIVE, task, Intervals.of(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) @@ -587,7 +636,7 @@ public void testUnlock() throws EntryExistsException taskStorage.insert(task, TaskStatus.running(task.getId())); lockbox.add(task); Assert.assertTrue( - lockbox.tryLock( + tryTimeChunkLock( TaskLockType.EXCLUSIVE, task, Intervals.of(StringUtils.format("2017-01-0%d/2017-01-0%d", (i + 1), (i + 2))) @@ -637,34 +686,363 @@ public void testFindLockPosseAfterRevokeWithDifferentLockIntervals() throws Entr lockbox.add(highPriorityTask); Assert.assertTrue( - lockbox.tryLock( + tryTimeChunkLock( TaskLockType.EXCLUSIVE, - lowPriorityTask, Intervals.of("2018-12-16T09:00:00/2018-12-16T10:00:00") + lowPriorityTask, + Intervals.of("2018-12-16T09:00:00/2018-12-16T10:00:00") ).isOk() ); Assert.assertTrue( - lockbox.tryLock( + tryTimeChunkLock( TaskLockType.EXCLUSIVE, - highPriorityTask, Intervals.of("2018-12-16T09:00:00/2018-12-16T09:30:00") + highPriorityTask, + Intervals.of("2018-12-16T09:00:00/2018-12-16T09:30:00") ).isOk() ); - final TaskLockPosse highLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval( + final List highLockPosses = lockbox.getOnlyTaskLockPosseContainingInterval( highPriorityTask, Intervals.of("2018-12-16T09:00:00/2018-12-16T09:30:00") ); - Assert.assertTrue(highLockPosse.containsTask(highPriorityTask)); - Assert.assertFalse(highLockPosse.getTaskLock().isRevoked()); + Assert.assertEquals(1, highLockPosses.size()); + Assert.assertTrue(highLockPosses.get(0).containsTask(highPriorityTask)); + Assert.assertFalse(highLockPosses.get(0).getTaskLock().isRevoked()); - final TaskLockPosse lowLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval( + final List lowLockPosses = lockbox.getOnlyTaskLockPosseContainingInterval( lowPriorityTask, Intervals.of("2018-12-16T09:00:00/2018-12-16T10:00:00") ); - Assert.assertTrue(lowLockPosse.containsTask(lowPriorityTask)); - Assert.assertTrue(lowLockPosse.getTaskLock().isRevoked()); + Assert.assertEquals(1, lowLockPosses.size()); + Assert.assertTrue(lowLockPosses.get(0).containsTask(lowPriorityTask)); + Assert.assertTrue(lowLockPosses.get(0).getTaskLock().isRevoked()); + } + + @Test + public void testSegmentLock() throws InterruptedException + { + final Task task = NoopTask.create(); + lockbox.add(task); + final LockResult lockResult = lockbox.lock( + task, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ); + Assert.assertTrue(lockResult.isOk()); + Assert.assertNull(lockResult.getNewSegmentId()); + Assert.assertTrue(lockResult.getTaskLock() instanceof SegmentLock); + final SegmentLock segmentLock = (SegmentLock) lockResult.getTaskLock(); + Assert.assertEquals(TaskLockType.EXCLUSIVE, segmentLock.getLockType()); + Assert.assertEquals(task.getGroupId(), segmentLock.getGroupId()); + Assert.assertEquals(task.getDataSource(), segmentLock.getDataSource()); + Assert.assertEquals(Intervals.of("2015-01-01/2015-01-02"), segmentLock.getInterval()); + Assert.assertEquals("v1", segmentLock.getVersion()); + Assert.assertEquals(3, segmentLock.getPartitionId()); + Assert.assertEquals(task.getPriority(), segmentLock.getPriority().intValue()); + Assert.assertFalse(segmentLock.isRevoked()); + } + + @Test + public void testSegmentAndTimeChunkLockForSameInterval() + { + final Task task1 = NoopTask.create(); + lockbox.add(task1); + + final Task task2 = NoopTask.create(); + lockbox.add(task2); + + Assert.assertTrue( + lockbox.tryLock( + task1, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task1, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ).isOk() + ); + + Assert.assertFalse( + lockbox.tryLock( + task2, + new TimeChunkLockRequest( + TaskLockType.EXCLUSIVE, + task2, + Intervals.of("2015-01-01/2015-01-02"), + "v1" + ) + ).isOk() + ); + } + + @Test + public void testSegmentAndTimeChunkLockForSameIntervalWithDifferentPriority() throws EntryExistsException + { + final Task task1 = NoopTask.create(10); + lockbox.add(task1); + taskStorage.insert(task1, TaskStatus.running(task1.getId())); + + final Task task2 = NoopTask.create(100); + lockbox.add(task2); + taskStorage.insert(task2, TaskStatus.running(task2.getId())); + + Assert.assertTrue( + lockbox.tryLock( + task1, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task1, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ).isOk() + ); + + Assert.assertTrue( + lockbox.tryLock( + task2, + new TimeChunkLockRequest( + TaskLockType.EXCLUSIVE, + task2, + Intervals.of("2015-01-01/2015-01-02"), + "v1" + ) + ).isOk() + ); + + final LockResult resultOfTask1 = lockbox.tryLock( + task1, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task1, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ); + Assert.assertFalse(resultOfTask1.isOk()); + Assert.assertTrue(resultOfTask1.isRevoked()); + } + + @Test + public void testLockWithDifferentGranularity() + { + final Task task = NoopTask.create("test", 10); + lockbox.add(task); + + Assert.assertTrue( + lockbox.tryLock( + task, + new TimeChunkLockRequest( + TaskLockType.EXCLUSIVE, + task, + Intervals.of("2015-01-01/2015-01-02"), + "v1" + ) + ).isOk() + ); + + Assert.assertFalse( + lockbox.tryLock( + task, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ).isOk() + ); + } + + @Test + public void testSegmentLockForSameIntervalAndSamePartition() + { + final Task task1 = NoopTask.create(); + lockbox.add(task1); + + final Task task2 = NoopTask.create(); + lockbox.add(task2); + + Assert.assertTrue( + lockbox.tryLock( + task1, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task1, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ).isOk() + ); + + Assert.assertFalse( + lockbox.tryLock( + task2, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task2, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ).isOk() + ); + } + + @Test + public void testSegmentLockForSameIntervalDifferentPartition() + { + final Task task1 = NoopTask.create(); + lockbox.add(task1); + + final Task task2 = NoopTask.create(); + lockbox.add(task2); + + Assert.assertTrue( + lockbox.tryLock( + task1, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task1, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 3 + ) + ).isOk() + ); + + Assert.assertTrue( + lockbox.tryLock( + task2, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task2, + Intervals.of("2015-01-01/2015-01-02"), + "v1", + 2 + ) + ).isOk() + ); + } + + @Test + public void testSegmentLockForOverlappedIntervalDifferentPartition() + { + final Task task1 = NoopTask.create(); + lockbox.add(task1); + + final Task task2 = NoopTask.create(); + lockbox.add(task2); + + Assert.assertTrue( + lockbox.tryLock( + task1, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task1, + Intervals.of("2015-01-01/2015-01-05"), + "v1", + 3 + ) + ).isOk() + ); + + Assert.assertFalse( + lockbox.tryLock( + task2, + new SpecificSegmentLockRequest( + TaskLockType.EXCLUSIVE, + task2, + Intervals.of("2015-01-03/2015-01-08"), + "v1", + 2 + ) + ).isOk() + ); + } + + @Test + public void testRequestForNewSegmentWithSegmentLock() + { + final Task task = NoopTask.create(); + lockbox.add(task); + allocateSegmentsAndAssert(task, "seq", 3, NumberedShardSpecFactory.instance()); + allocateSegmentsAndAssert(task, "seq2", 2, NumberedShardSpecFactory.instance()); //TODO ImmutableSet.of(0, 1, 2) for overwriting + + final List locks = lockbox.findLocksForTask(task); + Assert.assertEquals(5, locks.size()); + int expectedPartitionId = 0; + for (TaskLock lock : locks) { + Assert.assertTrue(lock instanceof SegmentLock); + final SegmentLock segmentLock = (SegmentLock) lock; + Assert.assertEquals(expectedPartitionId++, segmentLock.getPartitionId()); + } + } + + @Test + public void testRequestForNewSegmentWithHashPartition() + { + final Task task = NoopTask.create(); + lockbox.add(task); + + allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedShardSpecFactory(null, 3)); + allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedShardSpecFactory(null, 5)); + } + + private void allocateSegmentsAndAssert( + Task task, + String baseSequenceName, + int numSegmentsToAllocate, + ShardSpecFactory shardSpecFactory + ) + { + // TODO: test overshadowingSegments + for (int i = 0; i < numSegmentsToAllocate; i++) { + final LockRequestForNewSegment request = new LockRequestForNewSegment( + TaskLockType.EXCLUSIVE, + task, + Intervals.of("2015-01-01/2015-01-05"), + shardSpecFactory, + StringUtils.format("%s_%d", baseSequenceName, i), + null, + true + ); + assertAllocatedSegments(request, lockbox.tryLock(task, request)); + } + } + + private void assertAllocatedSegments( + LockRequestForNewSegment lockRequest, + LockResult result + ) + { + Assert.assertTrue(result.isOk()); + Assert.assertNotNull(result.getTaskLock()); + Assert.assertTrue(result.getTaskLock() instanceof SegmentLock); + Assert.assertNotNull(result.getNewSegmentId()); + final SegmentLock segmentLock = (SegmentLock) result.getTaskLock(); + final SegmentIdWithShardSpec segmentId = result.getNewSegmentId(); + + Assert.assertEquals(lockRequest.getType(), segmentLock.getLockType()); + Assert.assertEquals(lockRequest.getGroupId(), segmentLock.getGroupId()); + Assert.assertEquals(lockRequest.getDataSource(), segmentLock.getDataSource()); + Assert.assertEquals(lockRequest.getInterval(), segmentLock.getInterval()); + Assert.assertEquals(lockRequest.getShardSpecFactory().getShardSpecClass(), segmentId.getShardSpec().getClass()); + Assert.assertEquals(lockRequest.getPriority(), lockRequest.getPriority()); + // TODO: fix this to check overwriting shardSpec } @Test @@ -673,19 +1051,23 @@ public void testLockPosseEquals() final Task task1 = NoopTask.create(); final Task task2 = NoopTask.create(); - TaskLock taskLock1 = new TaskLock(TaskLockType.EXCLUSIVE, + TaskLock taskLock1 = new TimeChunkLock( + TaskLockType.EXCLUSIVE, task1.getGroupId(), task1.getDataSource(), Intervals.of("2018/2019"), "v1", - task1.getPriority()); + task1.getPriority() + ); - TaskLock taskLock2 = new TaskLock(TaskLockType.EXCLUSIVE, + TaskLock taskLock2 = new TimeChunkLock( + TaskLockType.EXCLUSIVE, task2.getGroupId(), task2.getDataSource(), Intervals.of("2018/2019"), "v2", - task2.getPriority()); + task2.getPriority() + ); TaskLockPosse taskLockPosse1 = new TaskLockPosse(taskLock1); TaskLockPosse taskLockPosse2 = new TaskLockPosse(taskLock2); @@ -704,24 +1086,24 @@ private Set getAllLocks(List tasks) .collect(Collectors.toSet()); } - private static class TaskLockWithoutPriority extends TaskLock + private static class IntervalLockWithoutPriority extends TimeChunkLock { @JsonCreator - TaskLockWithoutPriority( + IntervalLockWithoutPriority( String groupId, String dataSource, Interval interval, String version ) { - super(null, groupId, dataSource, interval, version, 0, false); + super(null, groupId, dataSource, interval, version, null, false); } @Override @JsonProperty - public TaskLockType getType() + public TaskLockType getLockType() { - return super.getType(); + return super.getLockType(); } @Override @@ -811,6 +1193,31 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 89824b046f35..20d77c732c7a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -41,6 +41,7 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; @@ -50,6 +51,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.Resource; +import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -63,6 +65,7 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; @@ -1022,6 +1025,31 @@ public boolean isReady(TaskActionClient taskActionClient) return false; } + @Override + public boolean requireLockInputSegments() + { + return false; + } + + @Override + public List findInputSegments(TaskActionClient taskActionClient, List intervals) + { + return Collections.emptyList(); + } + + @Override + public boolean changeSegmentGranularity(List intervalOfExistingSegments) + { + return false; + } + + @Nullable + @Override + public Granularity getSegmentGranularity(Interval interval) + { + return null; + } + @Override public TaskStatus run(TaskToolbox toolbox) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 0eeecd5375bd..1013f2210fb0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -19,15 +19,18 @@ package org.apache.druid.indexing.test; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.Interval; import java.util.ArrayList; @@ -37,6 +40,7 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator { + private final ObjectMapper objectMapper = new DefaultObjectMapper(); private final Set published = Sets.newConcurrentHashSet(); private final Set nuked = Sets.newConcurrentHashSet(); private final List unusedSegments; @@ -125,11 +129,17 @@ public SegmentIdWithShardSpec allocatePendingSegment( String sequenceName, String previousSegmentId, Interval interval, + ShardSpecFactory shardSpecFactory, String maxVersion, boolean skipSegmentLineageCheck ) { - throw new UnsupportedOperationException(); + return new SegmentIdWithShardSpec( + dataSource, + interval, + maxVersion, + shardSpecFactory.create(objectMapper, 0) + ); } @Override diff --git a/java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java b/java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java new file mode 100644 index 000000000000..1b37175e8e57 --- /dev/null +++ b/java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java @@ -0,0 +1,91 @@ +/* + * 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.java.util.common; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class IntRange +{ + private final int start; + private final int len; + + @JsonCreator + public IntRange( + @JsonProperty("start") int start, + @JsonProperty("len") int len + ) + { + this.start = start; + this.len = len; + } + + public boolean contains(int i) + { + return i >= start && i < getExclusiveEnd(); + } + + @JsonProperty + public int getStart() + { + return start; + } + + @JsonProperty + public int getLen() + { + return len; + } + + public int getExclusiveEnd() + { + return start + len; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IntRange intRange = (IntRange) o; + return start == intRange.start && + len == intRange.len; + } + + @Override + public int hashCode() + { + return Objects.hash(start, len); + } + + @Override + public String toString() + { + return "IntRange{" + + "start=" + start + + ", len=" + len + + '}'; + } +} diff --git a/java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java b/java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java new file mode 100644 index 000000000000..534ab3290b37 --- /dev/null +++ b/java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java @@ -0,0 +1,32 @@ +/* + * 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.java.util.common; + +import java.util.Comparator; + +public final class IntRanges +{ + public Comparator startThenEnd() + { + return Comparator.comparingInt(IntRange::getStart) + .thenComparing(range -> range.getStart() + range.getLen()); + } + + private IntRanges() {} +} diff --git a/pom.xml b/pom.xml index 4f752e119d65..168f7d39b303 100644 --- a/pom.xml +++ b/pom.xml @@ -1212,7 +1212,7 @@ - -Xmx1500m + -Xmx4g -XX:MaxDirectMemorySize=512m -Duser.language=en -Duser.GroupByQueryRunnerTest.javacountry=US diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java index ef98b7515b2e..5fe3dab05582 100644 --- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java @@ -19,7 +19,9 @@ package org.apache.druid.segment; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.timeline.Overshadowable; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -33,11 +35,15 @@ * until that. So ReferenceCountingSegment implements something like automatic reference count-based resource * management. */ -public class ReferenceCountingSegment extends AbstractSegment +public class ReferenceCountingSegment extends AbstractSegment implements Overshadowable { private static final EmittingLogger log = new EmittingLogger(ReferenceCountingSegment.class); private final Segment baseSegment; + private final short startRootPartitionId; + private final short endRootPartitionId; + private final short minorVersion; + private final short atomicUpdateGroupSize; private final AtomicBoolean closed = new AtomicBoolean(false); private final Phaser referents = new Phaser(1) { @@ -65,8 +71,29 @@ protected boolean onAdvance(int phase, int registeredParties) }; public ReferenceCountingSegment(Segment baseSegment) + { + this( + Preconditions.checkNotNull(baseSegment, "baseSegment"), + baseSegment.getId().getPartitionNum(), + (baseSegment.getId().getPartitionNum() + 1), + (short) 0, + (short) 1 + ); + } + + public ReferenceCountingSegment( + Segment baseSegment, + int startRootPartitionId, + int endRootPartitionId, + short minorVersion, + short atomicUpdateGroupSize + ) { this.baseSegment = baseSegment; + this.startRootPartitionId = (short) startRootPartitionId; + this.endRootPartitionId = (short) endRootPartitionId; + this.minorVersion = minorVersion; + this.atomicUpdateGroupSize = atomicUpdateGroupSize; } public Segment getBaseSegment() @@ -150,4 +177,50 @@ public T as(Class clazz) { return getBaseSegment().as(clazz); } + + @Override + public boolean isOvershadow(ReferenceCountingSegment other) + { + if (baseSegment.getId().getDataSource().equals(other.baseSegment.getId().getDataSource()) + && baseSegment.getId().getInterval().overlaps(other.baseSegment.getId().getInterval())) { + final int majorVersionCompare = baseSegment.getId().getVersion() + .compareTo(other.baseSegment.getId().getVersion()); + if (majorVersionCompare > 0) { + return true; + } else if (majorVersionCompare == 0) { + return includeRootPartitions(other) && getMinorVersion() > other.getMinorVersion(); + } + } + return false; + } + + private boolean includeRootPartitions(ReferenceCountingSegment other) + { + return startRootPartitionId <= other.startRootPartitionId + && endRootPartitionId >= other.endRootPartitionId; + } + + @Override + public int getStartRootPartitionId() + { + return startRootPartitionId; + } + + @Override + public int getEndRootPartitionId() + { + return endRootPartitionId; + } + + @Override + public short getMinorVersion() + { + return minorVersion; + } + + @Override + public short getAtomicUpdateGroupSize() + { + return atomicUpdateGroupSize; + } } diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java index 094e8fc2c321..cd7220e6c196 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java @@ -57,6 +57,7 @@ import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -435,7 +436,7 @@ public String toString() } public static QueryRunner makeFilteringQueryRunner( - final VersionedIntervalTimeline timeline, + final VersionedIntervalTimeline timeline, final QueryRunnerFactory> factory ) { @@ -453,7 +454,8 @@ public Sequence run(QueryPlus queryPlus, Map responseConte segments.addAll(timeline.lookup(interval)); } List> sequences = new ArrayList<>(); - for (TimelineObjectHolder holder : toolChest.filterSegments(query, segments)) { + // TODO: use other segment instead of ReferenceCountingSegment + for (TimelineObjectHolder holder : toolChest.filterSegments(query, segments)) { Segment segment = holder.getObject().getChunk(0).getObject(); QueryPlus queryPlusRunning = queryPlus.withQuerySegmentSpec( new SpecificSegmentSpec( diff --git a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java index 1b7eacc449f0..43a3af130dc0 100644 --- a/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/select/MultiSegmentSelectQueryTest.java @@ -42,6 +42,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.IncrementalIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -148,11 +149,10 @@ public static void setup() throws IOException segment1 = new IncrementalIndexSegment(index1, makeIdentifier(index1, "v1")); segment_override = new IncrementalIndexSegment(index2, makeIdentifier(index2, "v2")); - VersionedIntervalTimeline timeline = - new VersionedIntervalTimeline<>(StringComparators.LEXICOGRAPHIC); - timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk<>(segment0)); - timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk<>(segment1)); - timeline.add(index2.getInterval(), "v2", new SingleElementPartitionChunk<>(segment_override)); + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(StringComparators.LEXICOGRAPHIC); + timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk<>(new ReferenceCountingSegment(segment0))); + timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk<>(new ReferenceCountingSegment(segment1))); + timeline.add(index2.getInterval(), "v2", new SingleElementPartitionChunk<>(new ReferenceCountingSegment(segment_override))); segmentIdentifiers = new ArrayList<>(); for (TimelineObjectHolder holder : timeline.lookup(Intervals.of("2011-01-12/2011-01-14"))) { diff --git a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 769d5cf0e3e8..e999090fb9f8 100644 --- a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -34,6 +34,7 @@ import org.apache.druid.query.TableDataSource; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.IncrementalIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -144,10 +145,9 @@ private QueryRunner getCustomRunner() throws IOException segment0 = new IncrementalIndexSegment(index0, makeIdentifier(index0, "v1")); segment1 = new IncrementalIndexSegment(index1, makeIdentifier(index1, "v1")); - VersionedIntervalTimeline timeline = - new VersionedIntervalTimeline<>(StringComparators.LEXICOGRAPHIC); - timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk<>(segment0)); - timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk<>(segment1)); + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(StringComparators.LEXICOGRAPHIC); + timeline.add(index0.getInterval(), "v1", new SingleElementPartitionChunk<>(new ReferenceCountingSegment(segment0))); + timeline.add(index1.getInterval(), "v1", new SingleElementPartitionChunk<>(new ReferenceCountingSegment(segment1))); return QueryRunnerTestHelper.makeFilteringQueryRunner(timeline, factory); } diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java index 9229295cb414..5475ac3dc573 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.timeline.Overshadowable; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -475,14 +476,14 @@ private QueryableIndex makeAppendedMMappedIndex( List filesToMap = makeFilesToMap(tmpFile, files); - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( Comparators.naturalNullsFirst() ); ShardSpec noneShardSpec = NoneShardSpec.instance(); for (int i = 0; i < intervals.size(); i++) { - timeline.add(intervals.get(i), i, noneShardSpec.createChunk(filesToMap.get(i))); + timeline.add(intervals.get(i), i, noneShardSpec.createChunk(new OvershadowableFile(filesToMap.get(i)))); } final List adapters = Lists.newArrayList( @@ -490,23 +491,23 @@ private QueryableIndex makeAppendedMMappedIndex( // TimelineObjectHolder is actually an iterable of iterable of indexable adapters Iterables.transform( timeline.lookup(Intervals.of("1000-01-01/3000-01-01")), - new Function, Iterable>() + new Function, Iterable>() { @Override - public Iterable apply(final TimelineObjectHolder timelineObjectHolder) + public Iterable apply(final TimelineObjectHolder timelineObjectHolder) { return Iterables.transform( timelineObjectHolder.getObject(), // Each chunk can be used to build the actual IndexableAdapter - new Function, IndexableAdapter>() + new Function, IndexableAdapter>() { @Override - public IndexableAdapter apply(PartitionChunk chunk) + public IndexableAdapter apply(PartitionChunk chunk) { try { return new RowFilteringIndexAdapter( - new QueryableIndexIndexableAdapter(indexIO.loadIndex(chunk.getObject())), + new QueryableIndexIndexableAdapter(indexIO.loadIndex(chunk.getObject().file)), rowPointer -> timelineObjectHolder.getInterval().contains(rowPointer.getTimestamp()) ); } @@ -572,4 +573,44 @@ public QueryableIndex apply(@Nullable File input) throw new RuntimeException(e); } } + + private static class OvershadowableFile implements Overshadowable + { + private final File file; + + OvershadowableFile(File file) + { + this.file = file; + } + + @Override + public boolean isOvershadow(OvershadowableFile other) + { + return false; + } + + @Override + public int getStartRootPartitionId() + { + return 0; + } + + @Override + public int getEndRootPartitionId() + { + return 0; + } + + @Override + public short getMinorVersion() + { + return 0; + } + + @Override + public short getAtomicUpdateGroupSize() + { + return 0; + } + } } diff --git a/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java b/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java index a9cca5b154a5..83fd298485b3 100644 --- a/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java +++ b/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java @@ -23,10 +23,11 @@ import com.google.common.collect.Sets; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Overshadowable; import java.util.Set; -public class SegmentLoadInfo +public class SegmentLoadInfo implements Overshadowable { private final DataSegment segment; private final Set servers; @@ -94,4 +95,34 @@ public String toString() ", servers=" + servers + '}'; } + + @Override + public boolean isOvershadow(SegmentLoadInfo other) + { + return segment.isOvershadow(other.segment); + } + + @Override + public int getStartRootPartitionId() + { + return segment.getStartRootPartitionId(); + } + + @Override + public int getEndRootPartitionId() + { + return segment.getEndRootPartitionId(); + } + + @Override + public short getMinorVersion() + { + return segment.getMinorVersion(); + } + + @Override + public short getAtomicUpdateGroupSize() + { + return segment.getAtomicUpdateGroupSize(); + } } diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java index a485dbaa955c..0d1f9ba3a4ec 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java @@ -24,6 +24,7 @@ import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Overshadowable; import javax.annotation.Nullable; import java.util.ArrayList; @@ -35,7 +36,7 @@ /** */ -public class ServerSelector implements DiscoverySelector +public class ServerSelector implements DiscoverySelector, Overshadowable { private final Int2ObjectRBTreeMap> historicalServers; @@ -169,4 +170,36 @@ public QueryableDruidServer pick() return strategy.pick(realtimeServers, segment.get()); } } + + @Override + public boolean isOvershadow(ServerSelector other) + { + final DataSegment thisSegment = segment.get(); + final DataSegment thatSegment = other.getSegment(); + return thisSegment.isOvershadow(thatSegment); + } + + @Override + public int getStartRootPartitionId() + { + return segment.get().getStartRootPartitionId(); + } + + @Override + public int getEndRootPartitionId() + { + return segment.get().getEndRootPartitionId(); + } + + @Override + public short getMinorVersion() + { + return segment.get().getMinorVersion(); + } + + @Override + public short getAtomicUpdateGroupSize() + { + return segment.get().getAtomicUpdateGroupSize(); + } } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 95c1fce15c6e..979a660bf012 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -22,8 +22,10 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; import java.util.Collections; import java.util.List; @@ -103,8 +105,9 @@ default List getUsedSegmentsForInterval(String dataSource, Interval SegmentIdWithShardSpec allocatePendingSegment( String dataSource, String sequenceName, - String previousSegmentId, + @Nullable String previousSegmentId, Interval interval, + ShardSpecFactory shardSpecFactory, String maxVersion, boolean skipSegmentLineageCheck ); diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index c77dee153af0..c04021c0e503 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -44,10 +44,10 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.LinearShardSpec; 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.ShardSpecFactory; import org.joda.time.Interval; import org.skife.jdbi.v2.FoldController; import org.skife.jdbi.v2.Folder3; @@ -68,6 +68,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -345,6 +346,7 @@ public SegmentIdWithShardSpec allocatePendingSegment( final String sequenceName, @Nullable final String previousSegmentId, final Interval interval, + final ShardSpecFactory shardSpecFactory, final String maxVersion, final boolean skipSegmentLineageCheck ) @@ -352,12 +354,19 @@ public SegmentIdWithShardSpec allocatePendingSegment( Preconditions.checkNotNull(dataSource, "dataSource"); Preconditions.checkNotNull(sequenceName, "sequenceName"); Preconditions.checkNotNull(interval, "interval"); - Preconditions.checkNotNull(maxVersion, "maxVersion"); + Preconditions.checkNotNull(maxVersion, "version"); return connector.retryWithHandle( handle -> { if (skipSegmentLineageCheck) { - return allocatePendingSegment(handle, dataSource, sequenceName, interval, maxVersion); + return allocatePendingSegment( + handle, + dataSource, + sequenceName, + interval, + shardSpecFactory, + maxVersion + ); } else { return allocatePendingSegmentWithSegmentLineageCheck( handle, @@ -365,6 +374,7 @@ public SegmentIdWithShardSpec allocatePendingSegment( sequenceName, previousSegmentId, interval, + shardSpecFactory, maxVersion ); } @@ -379,6 +389,7 @@ private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( final String sequenceName, @Nullable final String previousSegmentId, final Interval interval, + final ShardSpecFactory shardSpecFactory, final String maxVersion ) throws IOException { @@ -406,7 +417,13 @@ private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( return result.segmentIdentifier; } - final SegmentIdWithShardSpec newIdentifier = createNewSegment(handle, dataSource, interval, maxVersion); + final SegmentIdWithShardSpec newIdentifier = createNewSegment( + handle, + dataSource, + interval, + shardSpecFactory, + maxVersion + ); if (newIdentifier == null) { return null; } @@ -446,6 +463,7 @@ private SegmentIdWithShardSpec allocatePendingSegment( final String dataSource, final String sequenceName, final Interval interval, + final ShardSpecFactory shardSpecFactory, final String maxVersion ) throws IOException { @@ -475,7 +493,13 @@ private SegmentIdWithShardSpec allocatePendingSegment( return result.segmentIdentifier; } - final SegmentIdWithShardSpec newIdentifier = createNewSegment(handle, dataSource, interval, maxVersion); + final SegmentIdWithShardSpec newIdentifier = createNewSegment( + handle, + dataSource, + interval, + shardSpecFactory, + maxVersion + ); if (newIdentifier == null) { return null; } @@ -614,13 +638,10 @@ private SegmentIdWithShardSpec createNewSegment( final Handle handle, final String dataSource, final Interval interval, + final ShardSpecFactory shardSpecFactory, final String maxVersion ) throws IOException { - // Make up a pending segment based on existing segments and pending segments in the DB. This works - // assuming that all tasks inserting segments at a particular point in time are going through the - // allocatePendingSegment flow. This should be assured through some other mechanism (like task locks). - final List> existingChunks = getTimelineForIntervalsWithHandle( handle, dataSource, @@ -630,24 +651,32 @@ private SegmentIdWithShardSpec createNewSegment( if (existingChunks.size() > 1) { // Not possible to expand more than one chunk with a single segment. log.warn( - "Cannot allocate new segment for dataSource[%s], interval[%s], maxVersion[%s]: already have [%,d] chunks.", + "Cannot allocate new segment for dataSource[%s], interval[%s]: already have [%,d] chunks.", dataSource, interval, - maxVersion, existingChunks.size() ); return null; + } else { + if (existingChunks + .stream() + .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) + .anyMatch(chunk -> !chunk.getObject().getShardSpec().isCompatible(shardSpecFactory.getShardSpecClass()))) { + // All existing segments should have a compatible shardSpec with shardSpecFactory. + return null; + } + SegmentIdWithShardSpec maxId = null; if (!existingChunks.isEmpty()) { TimelineObjectHolder existingHolder = Iterables.getOnlyElement(existingChunks); - for (PartitionChunk existing : existingHolder.getObject()) { - if (maxId == null || - maxId.getShardSpec().getPartitionNum() < existing.getObject().getShardSpec().getPartitionNum()) { - maxId = SegmentIdWithShardSpec.fromDataSegment(existing.getObject()); - } - } + + maxId = StreamSupport.stream(existingHolder.getObject().spliterator(), false) + .filter(chunk -> chunk.getObject().getShardSpec().getClass() == shardSpecFactory.getShardSpecClass()) + .max(Comparator.comparing(chunk -> chunk.getObject().getShardSpec().getPartitionNum())) + .map(chunk -> SegmentIdWithShardSpec.fromDataSegment(chunk.getObject())) + .orElse(null); } final List pendings = getPendingSegmentsForIntervalWithHandle( @@ -656,22 +685,35 @@ private SegmentIdWithShardSpec createNewSegment( interval ); - for (SegmentIdWithShardSpec pending : pendings) { - if (maxId == null || - pending.getVersion().compareTo(maxId.getVersion()) > 0 || - (pending.getVersion().equals(maxId.getVersion()) - && pending.getShardSpec().getPartitionNum() > maxId.getShardSpec().getPartitionNum())) { - maxId = pending; - } + if (maxId != null) { + pendings.add(maxId); + } + + maxId = pendings.stream() + .filter(id -> id.getShardSpec().getClass() == shardSpecFactory.getShardSpecClass()) + .max((id1, id2) -> { + final int versionCompare = id1.getVersion().compareTo(id2.getVersion()); + if (versionCompare != 0) { + return versionCompare; + } else { + return Integer.compare(id1.getShardSpec().getPartitionNum(), id2.getShardSpec().getPartitionNum()); + } + }) + .orElse(null); + + // Find the major version of existing segments + @Nullable final String versionOfExistingChunks; + if (!existingChunks.isEmpty()) { + versionOfExistingChunks = existingChunks.get(0).getVersion(); + } else if (!pendings.isEmpty()) { + versionOfExistingChunks = pendings.get(0).getVersion(); + } else { + versionOfExistingChunks = null; } if (maxId == null) { - return new SegmentIdWithShardSpec( - dataSource, - interval, - maxVersion, - new NumberedShardSpec(0, 0) - ); + final ShardSpec shardSpec = shardSpecFactory.create(jsonMapper, null); + return new SegmentIdWithShardSpec(dataSource, interval, versionOfExistingChunks == null ? maxVersion : versionOfExistingChunks, shardSpec); } else if (!maxId.getInterval().equals(interval) || maxId.getVersion().compareTo(maxVersion) > 0) { log.warn( "Cannot allocate new segment for dataSource[%s], interval[%s], maxVersion[%s]: conflicting segment[%s].", @@ -681,33 +723,14 @@ private SegmentIdWithShardSpec createNewSegment( maxId ); return null; - } else if (maxId.getShardSpec() instanceof LinearShardSpec) { - return new SegmentIdWithShardSpec( - dataSource, - maxId.getInterval(), - maxId.getVersion(), - new LinearShardSpec(maxId.getShardSpec().getPartitionNum() + 1) - ); - } else if (maxId.getShardSpec() instanceof NumberedShardSpec) { + } else { + final ShardSpec newShardSpec = shardSpecFactory.create(jsonMapper, maxId.getShardSpec()); return new SegmentIdWithShardSpec( dataSource, maxId.getInterval(), - maxId.getVersion(), - new NumberedShardSpec( - maxId.getShardSpec().getPartitionNum() + 1, - ((NumberedShardSpec) maxId.getShardSpec()).getPartitions() - ) - ); - } else { - log.warn( - "Cannot allocate new segment for dataSource[%s], interval[%s], maxVersion[%s]: ShardSpec class[%s] used by [%s].", - dataSource, - interval, - maxVersion, - maxId.getShardSpec().getClass(), - maxId + Preconditions.checkNotNull(versionOfExistingChunks, "versionOfExistingChunks"), + newShardSpec ); - return null; } } } @@ -751,7 +774,7 @@ private boolean announceHistoricalSegment( // SELECT -> INSERT can fail due to races; callers must be prepared to retry. // Avoiding ON DUPLICATE KEY since it's not portable. // Avoiding try/catch since it may cause inadvertent transaction-splitting. - handle.createStatement( + final int numRowsInserted = handle.createStatement( StringUtils.format( "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version, used, payload) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", @@ -770,7 +793,13 @@ private boolean announceHistoricalSegment( .bind("payload", jsonMapper.writeValueAsBytes(segment)) .execute(); - log.info("Published segment [%s] to DB with used flag [%s]", segment.getId(), used); + if (numRowsInserted == 1) { + log.info("Published segment [%s] to DB with used flag [%s], json[%s]", segment.getId(), used, jsonMapper.writeValueAsString(segment)); + } else if (numRowsInserted == 0) { + throw new ISE("Failed to publish segment[%s] to DB with used flag[%s], json[%s]", segment.getId(), used, jsonMapper.writeValueAsString(segment)); + } else { + throw new ISE("WTH? numRowsInserted[%s] is larger than 1 after inserting segment[%s] with used flag[%s], json[%s]", numRowsInserted, segment.getId(), used, jsonMapper.writeValueAsString(segment)); + } } catch (Exception e) { log.error(e, "Exception inserting segment [%s] with used flag [%s] into DB", segment.getId(), used); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 647340bbb502..f8c72f18c5b8 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -627,7 +627,11 @@ public ListenableFuture push( continue; } - final DataSegment dataSegment = mergeAndPush(entry.getKey(), entry.getValue(), useUniquePath); + final DataSegment dataSegment = mergeAndPush( + entry.getKey(), + entry.getValue(), + useUniquePath + ); if (dataSegment != null) { dataSegments.add(dataSegment); } else { @@ -746,7 +750,8 @@ private DataSegment mergeAndPush(final SegmentIdWithShardSpec identifier, final // semantics. () -> dataSegmentPusher.push( mergedFile, - sink.getSegment().withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes)), + sink.getSegment() + .withDimensions(IndexMerger.getMergedDimensionsFromQueryableIndexes(indexes)), useUniquePath ), exception -> exception instanceof Exception, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 44d91cb54416..d58772ade6a2 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -23,7 +23,6 @@ import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.IndexIO; @@ -33,8 +32,6 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.server.coordination.DataSegmentAnnouncer; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Interval; import java.util.concurrent.ExecutorService; @@ -127,9 +124,4 @@ public void unannounceSegments(Iterable segments) null ); } - - public static String getSequenceName(Interval interval, String version, ShardSpec shardSpec) - { - return StringUtils.format("index_%s_%s_%d", interval, version, shardSpec.getPartitionNum()); - } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index f8b12ad52ce3..7821b39663c6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -270,7 +270,7 @@ Map getSegments() * @return currently persisted commit metadata */ @Nullable - public abstract Object startJob(); + public abstract Object startJob(SegmentLocker segmentLocker); /** * Find a segment in the {@link SegmentState#APPENDING} state for the given timestamp and sequenceName. diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java index 3b084f067b5a..c38f853ef7d0 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java @@ -34,6 +34,7 @@ import java.io.IOException; import java.util.Collection; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -79,7 +80,7 @@ public BatchAppenderatorDriver( */ @Override @Nullable - public Object startJob() + public Object startJob(SegmentLocker segmentLocker) { final Object metadata = appenderator.startJob(); if (metadata != null) { @@ -129,11 +130,12 @@ private SegmentsAndMetadata pushAndClear( long pushAndClearTimeoutMs ) throws InterruptedException, ExecutionException, TimeoutException { - final Map requestedSegmentIdsForSequences = getAppendingSegments(sequenceNames) - .collect(Collectors.toMap(SegmentWithState::getSegmentIdentifier, Function.identity())); + final Set requestedSegmentIdsForSequences = getAppendingSegments(sequenceNames) + .map(SegmentWithState::getSegmentIdentifier) + .collect(Collectors.toSet()); final ListenableFuture future = ListenableFutures.transformAsync( - pushInBackground(null, requestedSegmentIdsForSequences.keySet(), false), + pushInBackground(null, requestedSegmentIdsForSequences, false), this::dropInBackground ); @@ -147,11 +149,11 @@ private SegmentsAndMetadata pushAndClear( .stream() .collect(Collectors.toMap(SegmentIdWithShardSpec::fromDataSegment, Function.identity())); - if (!pushedSegmentIdToSegmentMap.keySet().equals(requestedSegmentIdsForSequences.keySet())) { + if (!pushedSegmentIdToSegmentMap.keySet().equals(requestedSegmentIdsForSequences)) { throw new ISE( "Pushed segments[%s] are different from the requested ones[%s]", pushedSegmentIdToSegmentMap.keySet(), - requestedSegmentIdsForSequences.keySet() + requestedSegmentIdsForSequences ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java index 58dad0d36448..6e85794e69d1 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java @@ -59,6 +59,16 @@ public SegmentId asSegmentId() return id; } + public SegmentIdWithShardSpec withShardSpec(ShardSpec shardSpec) + { + return new SegmentIdWithShardSpec( + id.getDataSource(), + id.getInterval(), + id.getVersion(), + shardSpec + ); + } + @JsonProperty public String getDataSource() { @@ -83,6 +93,11 @@ public ShardSpec getShardSpec() return shardSpec; } + public String getIdentifierAsString() + { + return asString; + } + @Override public boolean equals(Object o) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentLocker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentLocker.java new file mode 100644 index 000000000000..6c5bcba3381d --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentLocker.java @@ -0,0 +1,25 @@ +/* + * 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.segment.realtime.appenderator; + +public interface SegmentLocker +{ + boolean lock(SegmentIdWithShardSpec segmentId); +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index 2599387dd158..ea98d87d1f68 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -107,7 +107,7 @@ public StreamAppenderatorDriver( @Override @Nullable - public Object startJob() + public Object startJob(SegmentLocker segmentLocker) { handoffNotifier.start(); @@ -133,6 +133,14 @@ public Object startJob() final SegmentsForSequenceBuilder builder = new SegmentsForSequenceBuilder(lastSegmentIds.get(sequenceName)); builders.put(sequenceName, builder); entry.getValue().forEach(builder::add); + if (segmentLocker != null) { + for (SegmentWithState segmentWithState : entry.getValue()) { + if (segmentWithState.getState() != SegmentState.PUSHED_AND_DROPPED + && !segmentLocker.lock(segmentWithState.getSegmentIdentifier())) { + throw new ISE("Failed to lock segment[%s]", segmentWithState.getSegmentIdentifier()); + } + } + } } builders.forEach((sequence, builder) -> segments.put(sequence, builder.build())); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java index 0749a2892036..0ab3ed2d35e4 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -20,11 +20,21 @@ package org.apache.druid.segment.realtime.appenderator; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.OverwritingShardSpec; +import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.Set; +import java.util.stream.Collectors; public interface TransactionalSegmentPublisher { @@ -38,8 +48,62 @@ public interface TransactionalSegmentPublisher * @throws IOException if there was an I/O error when publishing * @throws RuntimeException if we cannot tell if the segments were published or not, for some other reason */ - SegmentPublishResult publishSegments( + SegmentPublishResult publishAnnotatedSegments( Set segments, @Nullable Object commitMetadata ) throws IOException; + + default SegmentPublishResult publishSegments(Set segments, @Nullable Object commitMetadata) + throws IOException + { + return publishAnnotatedSegments(annotateAtomicUpdateGroupSize(segments), commitMetadata); + } + + static Set annotateAtomicUpdateGroupSize(Set segments) + { + final Map> intervalToSegments = new HashMap<>(); + segments.forEach( + segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + + for (Entry> entry : intervalToSegments.entrySet()) { + final Interval interval = entry.getKey(); + final List segmentsPerInterval = entry.getValue(); + final boolean isNonFirstGeneration = segmentsPerInterval.get(0).getShardSpec() instanceof OverwritingShardSpec; + + final boolean anyMismatch = segmentsPerInterval.stream().anyMatch( + segment -> (segment.getShardSpec() instanceof OverwritingShardSpec) != isNonFirstGeneration + ); + if (anyMismatch) { + throw new ISE( + "WTH? some segments have empty overshadwedSegments but others are not? " + + "segments with non-overwritingShardSpec: [%s]," + + "segments with overwritingShardSpec: [%s]", + segmentsPerInterval.stream() + .filter(segment -> !(segment.getShardSpec() instanceof OverwritingShardSpec)) + .collect(Collectors.toList()), + segmentsPerInterval.stream() + .filter(segment -> segment.getShardSpec() instanceof OverwritingShardSpec) + .collect(Collectors.toList()) + ); + } + + if (isNonFirstGeneration) { + // The segments which are published together consist an atomicUpdateGroup. + + intervalToSegments.put( + interval, + segmentsPerInterval + .stream() + .map(segment -> { + final OverwritingShardSpec shardSpec = (OverwritingShardSpec) segment.getShardSpec(); + return segment.withShardSpec(shardSpec.withAtomicUpdateGroupSize((short) segmentsPerInterval.size())); + }) + .collect(Collectors.toList()) + ); + } + } + + return intervalToSegments.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()); + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index 8e0ad2b999f5..005f167a1830 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -737,7 +737,7 @@ private void addSink(final Sink sink) sinkTimeline.add( sink.getInterval(), sink.getVersion(), - new SingleElementPartitionChunk(sink) + new SingleElementPartitionChunk<>(sink) ); try { segmentAnnouncer.announceSegment(sink.getSegment()); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index d2d72bacfa6d..67506d7b5c1e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Overshadowable; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; @@ -52,7 +53,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; -public class Sink implements Iterable +public class Sink implements Iterable, Overshadowable { private static final IncrementalIndexAddResult ALREADY_SWAPPED = new IncrementalIndexAddResult(-1, -1, null, "write after index swapped"); @@ -407,4 +408,36 @@ public String toString() ", schema=" + schema + '}'; } + + @Override + public boolean isOvershadow(Sink other) + { + // Sink is currently used in timeline only for querying stream data. + // In this case, sinks never overshadow each other. + 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(); + } } diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index ecbb0d496861..5fde55d0c274 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -32,6 +32,7 @@ import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; +import org.apache.druid.timeline.partition.ShardSpec; import javax.annotation.Nullable; import java.util.Map; @@ -174,10 +175,11 @@ public boolean loadSegment(final DataSegment segment) throws SegmentLoadingExcep log.warn("Told to load an adapter for segment[%s] that already exists", segment.getId()); resultSupplier.set(false); } else { + final ReferenceCountingSegment referenceCountingSegment = new ReferenceCountingSegment(adapter); loadedIntervals.add( segment.getInterval(), segment.getVersion(), - segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter)) + segment.getShardSpec().createChunk(referenceCountingSegment) ); dataSourceState.addSegment(segment); resultSupplier.set(true); @@ -220,12 +222,21 @@ public void dropSegment(final DataSegment segment) final VersionedIntervalTimeline loadedIntervals = dataSourceState.getTimeline(); + final ShardSpec shardSpec = segment.getShardSpec(); final PartitionChunk removed = loadedIntervals.remove( segment.getInterval(), segment.getVersion(), // remove() internally searches for a partitionChunk to remove which is *equal* to the given // partitionChunk. Note that partitionChunk.equals() checks only the partitionNum, but not the object. - segment.getShardSpec().createChunk(null) + segment.getShardSpec().createChunk( + new ReferenceCountingSegment( + null, + shardSpec.getStartRootPartitionId(), + shardSpec.getEndRootPartitionId(), + shardSpec.getMinorVersion(), + shardSpec.getAtomicUpdateGroupSize() + ) + ) ); final ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index 8131e4f62be9..660ec7029ae6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -73,7 +73,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) //Remove all segments in db that are overshadowed by served segments for (DataSegment dataSegment : params.getAvailableSegments()) { VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { + if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion(), dataSegment)) { coordinator.removeSegment(dataSegment); stats.addToGlobalStat("overShadowedCount", 1); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index 7d7830170c99..e82aba52e0d2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -150,7 +150,7 @@ private Set determineOvershadowedSegments(DruidCoordinatorRuntimePa Set overshadowed = new HashSet<>(); for (VersionedIntervalTimeline timeline : timelines.values()) { - for (TimelineObjectHolder holder : timeline.findOvershadowed()) { + for (TimelineObjectHolder holder : timeline.findFullyOvershadowed()) { for (DataSegment dataSegment : holder.getObject().payloads()) { overshadowed.add(dataSegment); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index ca3a309e687e..7dd7847022f7 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -1763,11 +1763,15 @@ private ServerSelector makeMockSingleDimensionSelector( int partitionNum ) { - DataSegment segment = EasyMock.createNiceMock(DataSegment.class); - EasyMock.expect(segment.getId()).andReturn(SegmentId.dummy(DATA_SOURCE)).anyTimes(); - EasyMock.expect(segment.getShardSpec()).andReturn(new SingleDimensionShardSpec(dimension, start, end, partitionNum)) - .anyTimes(); - EasyMock.replay(segment); + final DataSegment segment = new DataSegment( + SegmentId.dummy(DATA_SOURCE), + null, + null, + null, + new SingleDimensionShardSpec(dimension, start, end, partitionNum), + 9, + 0L + ); ServerSelector selector = new ServerSelector( segment, @@ -2204,11 +2208,26 @@ private List> populateTimeline( serverExpectations.put(lastServer, new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class))); } + final ShardSpec shardSpec; + if (numChunks == 1) { + shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0); + } else { + String start = null; + String end = null; + if (j > 0) { + start = String.valueOf(j); + } + if (j + 1 < numChunks) { + end = String.valueOf(j + 1); + } + shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j); + } DataSegment mockSegment = makeMock(mocks, DataSegment.class); ServerExpectation expectation = new ServerExpectation<>( SegmentId.dummy(StringUtils.format("%s_%s", k, j)), // interval/chunk queryIntervals.get(k), mockSegment, + shardSpec, expectedResults.get(k).get(j) ); serverExpectations.get(lastServer).addExpectation(expectation); @@ -2219,20 +2238,6 @@ private List> populateTimeline( new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()) ); selector.addServerAndUpdateSegment(new QueryableDruidServer(lastServer, null), selector.getSegment()); - final ShardSpec shardSpec; - if (numChunks == 1) { - shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0); - } else { - String start = null; - String end = null; - if (j > 0) { - start = String.valueOf(j); - } - if (j + 1 < numChunks) { - end = String.valueOf(j + 1); - } - shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j); - } EasyMock.reset(mockSegment); EasyMock.expect(mockSegment.getShardSpec()) .andReturn(shardSpec) @@ -2746,18 +2751,21 @@ private static class ServerExpectation private final SegmentId segmentId; private final Interval interval; private final DataSegment segment; + private final ShardSpec shardSpec; private final Iterable> results; public ServerExpectation( SegmentId segmentId, Interval interval, DataSegment segment, + ShardSpec shardSpec, Iterable> results ) { this.segmentId = segmentId; this.interval = interval; this.segment = segment; + this.shardSpec = shardSpec; this.results = results; } @@ -2901,6 +2909,43 @@ public String toString() { return baseSegment.toString(); } + + @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(); + } + + @Override + public boolean isOvershadow(DataSegment other) + { + if (getDataSource().equals(other.getDataSource()) + && getInterval().overlaps(other.getInterval()) + && getVersion().equals(other.getVersion())) { + return getStartRootPartitionId() <= other.getStartRootPartitionId() + && getEndRootPartitionId() >= other.getEndRootPartitionId() + && getMinorVersion() > other.getMinorVersion(); + } + return false; + } } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 879ddfb13b4c..e7da678c4518 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -32,9 +32,18 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwritingShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpecFactory; +import org.apache.druid.timeline.partition.PartitionChunk; +import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -48,10 +57,12 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; public class IndexerSQLMetadataStorageCoordinatorTest { @@ -181,7 +192,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest public void setUp() { derbyConnector = derbyConnectorRule.getConnector(); - mapper.registerSubtypes(LinearShardSpec.class, NumberedShardSpec.class); + mapper.registerSubtypes(LinearShardSpec.class, NumberedShardSpec.class, HashBasedNumberedShardSpec.class); derbyConnector.createDataSourceTable(); derbyConnector.createTaskTables(); derbyConnector.createSegmentTable(); @@ -840,6 +851,7 @@ private void additionalNumberedShardTest(Set segments) throws IOExc @Test public void testAllocatePendingSegment() { + final ShardSpecFactory shardSpecFactory = NumberedShardSpecFactory.instance(); final String dataSource = "ds"; final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( @@ -847,6 +859,7 @@ public void testAllocatePendingSegment() "seq", null, interval, + shardSpecFactory, "version", false ); @@ -858,6 +871,7 @@ public void testAllocatePendingSegment() "seq", identifier.toString(), interval, + shardSpecFactory, identifier.getVersion(), false ); @@ -869,6 +883,7 @@ public void testAllocatePendingSegment() "seq", identifier1.toString(), interval, + shardSpecFactory, identifier1.getVersion(), false ); @@ -880,6 +895,7 @@ public void testAllocatePendingSegment() "seq", identifier1.toString(), interval, + shardSpecFactory, identifier1.getVersion(), false ); @@ -892,6 +908,7 @@ public void testAllocatePendingSegment() "seq1", null, interval, + shardSpecFactory, "version", false ); @@ -902,6 +919,7 @@ public void testAllocatePendingSegment() @Test public void testDeletePendingSegment() throws InterruptedException { + final ShardSpecFactory shardSpecFactory = NumberedShardSpecFactory.instance(); final String dataSource = "ds"; final Interval interval = Intervals.of("2017-01-01/2017-02-01"); String prevSegmentId = null; @@ -914,6 +932,7 @@ public void testDeletePendingSegment() throws InterruptedException "seq", prevSegmentId, interval, + shardSpecFactory, "version", false ); @@ -928,6 +947,7 @@ public void testDeletePendingSegment() throws InterruptedException "seq", prevSegmentId, interval, + shardSpecFactory, "version", false ); @@ -937,4 +957,166 @@ public void testDeletePendingSegment() throws InterruptedException final int numDeleted = coordinator.deletePendingSegments(dataSource, new Interval(begin, secondBegin)); Assert.assertEquals(10, numDeleted); } + + // TODO: fix this test to use overwriting shardSpec + @Test + public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOException + { + final String dataSource = "ds"; + final Interval interval = Intervals.of("2017-01-01/2017-02-01"); + String prevSegmentId = null; + + for (int i = 0; i < 10; i++) { + final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( + dataSource, + "seq", + prevSegmentId, + interval, + new NumberedOverwritingShardSpecFactory(0, 1, (short) (i + 1)), + "version", + false + ); + Assert.assertEquals( + StringUtils.format( + "ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version%s", + "_" + (i + ShardSpec.NON_ROOT_GEN_START_PARTITION_ID) + ), + identifier.toString() + ); + prevSegmentId = identifier.toString(); + final Set toBeAnnounced = Collections.singleton( + new DataSegment( + identifier.getDataSource(), + identifier.getInterval(), + identifier.getVersion(), + null, + Collections.emptyList(), + Collections.emptyList(), + ((NumberedOverwritingShardSpec) identifier.getShardSpec()).withAtomicUpdateGroupSize(1), + 0, + 10L + ) + ); + final Set announced = coordinator.announceHistoricalSegments(toBeAnnounced); + + Assert.assertEquals(toBeAnnounced, announced); + } + + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline + .forSegments(coordinator.getUsedSegmentsForInterval(dataSource, interval)); + + final List visibleSegments = timeline + .lookup(interval) + .stream() + .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); + + Assert.assertEquals(1, visibleSegments.size()); + Assert.assertEquals( + new DataSegment( + dataSource, + interval, + "version", + null, + Collections.emptyList(), + Collections.emptyList(), + new NumberedOverwritingShardSpec( + 9 + ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, + 0, + 1, + (short) 9, + (short) 1 + ), + 0, + 10L + ), + visibleSegments.get(0) + ); + } + + @Test + public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IOException + { + final ShardSpecFactory shardSpecFactory = new HashBasedNumberedShardSpecFactory( + null, + 5 + ); + final String dataSource = "ds"; + final Interval interval = Intervals.of("2017-01-01/2017-02-01"); + + SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( + dataSource, + "seq", + null, + interval, + shardSpecFactory, + "version", + true + ); + + HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); + Assert.assertEquals(0, shardSpec.getPartitionNum()); + Assert.assertEquals(5, shardSpec.getPartitions()); + + coordinator.announceHistoricalSegments( + Collections.singleton( + new DataSegment( + id.getDataSource(), + id.getInterval(), + id.getVersion(), + null, + Collections.emptyList(), + Collections.emptyList(), + id.getShardSpec(), + 0, + 10L + ) + ) + ); + + id = coordinator.allocatePendingSegment( + dataSource, + "seq2", + null, + interval, + shardSpecFactory, + "version", + true + ); + + shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); + Assert.assertEquals(1, shardSpec.getPartitionNum()); + Assert.assertEquals(5, shardSpec.getPartitions()); + + coordinator.announceHistoricalSegments( + Collections.singleton( + new DataSegment( + id.getDataSource(), + id.getInterval(), + id.getVersion(), + null, + Collections.emptyList(), + Collections.emptyList(), + id.getShardSpec(), + 0, + 10L + ) + ) + ); + + id = coordinator.allocatePendingSegment( + dataSource, + "seq3", + null, + interval, + new HashBasedNumberedShardSpecFactory(null, 3), + "version", + true + ); + + shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); + Assert.assertEquals(2, shardSpec.getPartitionNum()); + Assert.assertEquals(3, shardSpec.getPartitions()); + } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java index 6536cb6d4a30..a484ba2117cf 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java @@ -103,7 +103,7 @@ public void tearDown() throws Exception @Test public void testSimple() throws Exception { - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (InputRow row : ROWS) { Assert.assertTrue(driver.add(row, "dummy").isOk()); @@ -137,7 +137,7 @@ public void testSimple() throws Exception @Test public void testIncrementalPush() throws Exception { - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); int i = 0; for (InputRow row : ROWS) { @@ -174,11 +174,11 @@ public void testIncrementalPush() throws Exception @Test public void testRestart() { - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); driver.close(); appenderatorTester.getAppenderator().close(); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); } private void checkSegmentStates(int expectedNumSegmentsInState, SegmentState expectedState) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java index 9d922537159b..bbbc64a99041 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverFailTest.java @@ -137,12 +137,12 @@ public void testFailDuringPersist() throws IOException, InterruptedException, Ti new FireDepartmentMetrics() ); - driver.startJob(); + driver.startJob(null); final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); segmentHandoffNotifierFactory.setHandoffDelay(100); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); @@ -175,12 +175,12 @@ public void testFailDuringPush() throws IOException, InterruptedException, Timeo new FireDepartmentMetrics() ); - driver.startJob(); + driver.startJob(null); final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); segmentHandoffNotifierFactory.setHandoffDelay(100); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); @@ -213,12 +213,12 @@ public void testFailDuringDrop() throws IOException, InterruptedException, Timeo new FireDepartmentMetrics() ); - driver.startJob(); + driver.startJob(null); final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); segmentHandoffNotifierFactory.setHandoffDelay(100); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); @@ -266,12 +266,12 @@ private void testFailDuringPublishInternal(boolean failWithException) throws Exc new FireDepartmentMetrics() ); - driver.startJob(); + driver.startJob(null); final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); segmentHandoffNotifierFactory.setHandoffDelay(100); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java index 9c3e9ccbe4ca..c544eb6c8314 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java @@ -133,7 +133,7 @@ public void testSimple() throws Exception { final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); @@ -169,7 +169,7 @@ public void testMaxRowsPerSegment() throws Exception { final int numSegments = 3; final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (int i = 0; i < numSegments * MAX_ROWS_PER_SEGMENT; i++) { committerSupplier.setMetadata(i + 1); @@ -212,7 +212,7 @@ public void testHandoffTimeout() throws Exception final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); segmentHandoffNotifierFactory.disableHandoff(); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); for (int i = 0; i < ROWS.size(); i++) { committerSupplier.setMetadata(i + 1); @@ -237,7 +237,7 @@ public void testPublishPerRow() throws IOException, InterruptedException, Timeou { final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); // Add the first row and publish immediately { @@ -305,7 +305,7 @@ public void testIncrementalHandoff() throws Exception { final TestCommitterSupplier committerSupplier = new TestCommitterSupplier<>(); - Assert.assertNull(driver.startJob()); + Assert.assertNull(driver.startJob(null)); committerSupplier.setMetadata(1); Assert.assertTrue(driver.add(ROWS.get(0), "sequence_0", committerSupplier, false, true).isOk()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java index ffa9e9bfd0d4..6e0607635e91 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java @@ -662,7 +662,7 @@ private static VersionedIntervalTimeline createTimeline( } for (int i = 0; i < spec.numSegmentsPerShard; i++) { - final ShardSpec shardSpec = new NumberedShardSpec(spec.numSegmentsPerShard, i); + final ShardSpec shardSpec = new NumberedShardSpec(i, spec.numSegmentsPerShard); final DataSegment segment = new DataSegment( DATA_SOURCE, segmentInterval, diff --git a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java index 497a6d73a77b..7192c61a93a0 100644 --- a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java @@ -26,8 +26,10 @@ import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.ServerTestHelper; +import org.apache.druid.timeline.Overshadowable; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.NumberedOverwritingShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; @@ -38,6 +40,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Objects; import java.util.Set; public class NumberedShardSpecTest @@ -113,11 +116,14 @@ public PartitionChunk apply(ShardSpec shardSpec) public void testVersionedIntervalTimelineBehaviorForNumberedShardSpec() { //core partition chunks - PartitionChunk chunk0 = new NumberedShardSpec(0, 2).createChunk("0"); - PartitionChunk chunk1 = new NumberedShardSpec(1, 2).createChunk("1"); + PartitionChunk chunk0 = new NumberedShardSpec(0, 2) + .createChunk(new OvershadowableString("0", 0)); + PartitionChunk chunk1 = new NumberedShardSpec(1, 2) + .createChunk(new OvershadowableString("1", 1)); //appended partition chunk - PartitionChunk chunk4 = new NumberedShardSpec(4, 2).createChunk("4"); + PartitionChunk chunk4 = new NumberedShardSpec(4, 2) + .createChunk(new OvershadowableString("4", 4)); //incomplete partition sets testVersionedIntervalTimelineBehaviorForNumberedShardSpec( @@ -150,53 +156,136 @@ public void testVersionedIntervalTimelineBehaviorForNumberedShardSpec() //complete partition sets testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk1, chunk0), - ImmutableSet.of("0", "1") + ImmutableSet.of(new OvershadowableString("0", 0), new OvershadowableString("1", 1)) ); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk4, chunk1, chunk0), - ImmutableSet.of("0", "1", "4") + ImmutableSet.of( + new OvershadowableString("0", 0), + new OvershadowableString("1", 1), + new OvershadowableString("4", 4) + ) ); // a partition set with 0 core partitions - chunk0 = new NumberedShardSpec(0, 0).createChunk("0"); - chunk4 = new NumberedShardSpec(4, 0).createChunk("4"); + chunk0 = new NumberedShardSpec(0, 0).createChunk(new OvershadowableString("0", 0)); + chunk4 = new NumberedShardSpec(4, 0).createChunk(new OvershadowableString("4", 4)); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk0), - ImmutableSet.of("0") + ImmutableSet.of(new OvershadowableString("0", 0)) ); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk4), - ImmutableSet.of("4") + ImmutableSet.of(new OvershadowableString("4", 4)) ); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk4, chunk0), - ImmutableSet.of("0", "4") + ImmutableSet.of(new OvershadowableString("0", 0), new OvershadowableString("4", 4)) ); } private void testVersionedIntervalTimelineBehaviorForNumberedShardSpec( - List> chunks, - Set expectedObjects + List> chunks, + Set expectedObjects ) { - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); Interval interval = Intervals.of("2000/3000"); String version = "v1"; - for (PartitionChunk chunk : chunks) { + for (PartitionChunk chunk : chunks) { timeline.add(interval, version, chunk); } - Set actualObjects = new HashSet<>(); - List> entries = timeline.lookup(interval); - for (TimelineObjectHolder entry : entries) { - for (PartitionChunk chunk : entry.getObject()) { + Set actualObjects = new HashSet<>(); + List> entries = timeline.lookup(interval); + for (TimelineObjectHolder entry : entries) { + for (PartitionChunk chunk : entry.getObject()) { actualObjects.add(chunk.getObject()); } } Assert.assertEquals(expectedObjects, actualObjects); } + + @Test + public void testCompatible() + { + final NumberedShardSpec spec = new NumberedShardSpec(0, 0); + Assert.assertTrue(spec.isCompatible(NumberedShardSpec.class)); + Assert.assertTrue(spec.isCompatible(NumberedOverwritingShardSpec.class)); + } + + private static final class OvershadowableString implements Overshadowable + { + private final int partitionId; + private final String val; + + OvershadowableString(String val, int partitionId) + { + this.val = val; + this.partitionId = partitionId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OvershadowableString that = (OvershadowableString) o; + return partitionId == that.partitionId && + Objects.equals(val, that.val); + } + + @Override + public int hashCode() + { + return Objects.hash(partitionId, val); + } + + @Override + public String toString() + { + return "OvershadowableString{" + + "partitionId=" + partitionId + + ", val='" + val + '\'' + + '}'; + } + + @Override + public boolean isOvershadow(OvershadowableString other) + { + return false; + } + + @Override + public int getStartRootPartitionId() + { + return partitionId; + } + + @Override + public int getEndRootPartitionId() + { + return partitionId + 1; + } + + @Override + public short getMinorVersion() + { + return 0; + } + + @Override + public short getAtomicUpdateGroupSize() + { + return 1; + } + } } diff --git a/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java b/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java index 61e58409a428..86b622a60ee1 100644 --- a/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java @@ -132,7 +132,6 @@ public void testIsInChunk() specs.add(new HashOverridenShardSpec(i, 3)); } - assertExistsInOneSpec(specs, new HashInputRow(Integer.MIN_VALUE)); assertExistsInOneSpec(specs, new HashInputRow(Integer.MAX_VALUE)); assertExistsInOneSpec(specs, new HashInputRow(0)); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index d354f141d9af..9ad78af4063b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -257,8 +257,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), null, 1, - 53000L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 53000L ); private final DataSegment publishedSegment2 = new DataSegment( "wikipedia2", @@ -269,8 +268,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), null, 1, - 83000L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 83000L ); private final DataSegment publishedSegment3 = new DataSegment( "wikipedia3", @@ -281,8 +279,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), null, 1, - 47000L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 47000L ); private final DataSegment segment1 = new DataSegment( @@ -294,8 +291,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), null, 1, - 100L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 100L ); private final DataSegment segment2 = new DataSegment( "test2", @@ -306,8 +302,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), null, 1, - 100L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 100L ); private final DataSegment segment3 = new DataSegment( "test3", @@ -318,8 +313,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), new NumberedShardSpec(2, 3), 1, - 100L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 100L ); private final DataSegment segment4 = new DataSegment( "test4", @@ -330,8 +324,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), null, 1, - 100L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 100L ); private final DataSegment segment5 = new DataSegment( "test5", @@ -342,8 +335,7 @@ public Authorizer getAuthorizer(String name) ImmutableList.of("met1", "met2"), null, 1, - 100L, - DataSegment.PruneLoadSpecHolder.DEFAULT + 100L ); final List realtimeSegments = ImmutableList.of(segment2, segment4, segment5); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index af0e89454328..527f4acb0ce1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -47,6 +47,7 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -63,7 +64,7 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, Closeable { private final QueryRunnerFactoryConglomerate conglomerate; - private final Map> timelines = new HashMap<>(); + private final Map> timelines = new HashMap<>(); private final List closeables = new ArrayList<>(); private final List segments = new ArrayList<>(); @@ -82,8 +83,8 @@ public SpecificSegmentsQuerySegmentWalker add( timelines.put(descriptor.getDataSource(), new VersionedIntervalTimeline<>(Ordering.natural())); } - final VersionedIntervalTimeline timeline = timelines.get(descriptor.getDataSource()); - timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment)); + final VersionedIntervalTimeline timeline = timelines.get(descriptor.getDataSource()); + timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(new ReferenceCountingSegment(segment))); segments.add(descriptor); closeables.add(index); return this; @@ -135,7 +136,7 @@ public QueryRunner getQueryRunnerForIntervals( )) .build(); } - final VersionedIntervalTimeline timeline = getTimelineForTableDataSource( + final VersionedIntervalTimeline timeline = getTimelineForTableDataSource( newQuery1); return makeBaseRunner( newQuery1, @@ -199,7 +200,7 @@ public void close() throws IOException } } - private VersionedIntervalTimeline getTimelineForTableDataSource(Query query) + private VersionedIntervalTimeline getTimelineForTableDataSource(Query query) { if (query.getDataSource() instanceof TableDataSource) { return timelines.get(((TableDataSource) query.getDataSource()).getName()); @@ -215,7 +216,7 @@ private QueryRunner makeBaseRunner( final Iterable specs ) { - final VersionedIntervalTimeline timeline = getTimelineForTableDataSource(query); + final VersionedIntervalTimeline timeline = getTimelineForTableDataSource(query); if (timeline == null) { return new NoopQueryRunner<>(); } @@ -228,7 +229,7 @@ private QueryRunner makeBaseRunner( .create(specs) .transformCat( descriptor -> { - final PartitionHolder holder = timeline.findEntry( + final PartitionHolder holder = timeline.findEntry( descriptor.getInterval(), descriptor.getVersion() ); From 9f246048af9a1be96cc52c74b17a25373bc24818 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Apr 2019 09:58:35 -0700 Subject: [PATCH 02/52] Allow both timeChunk and segment lock in the same gruop --- .../indexing/common/actions/TaskAction.java | 1 + .../druid/indexing/common/task/NoopTask.java | 16 +- .../druid/indexing/overlord/TaskLockbox.java | 184 +++++++++--------- .../actions/RemoteTaskActionClientTest.java | 4 +- .../actions/SegmentAllocateActionTest.java | 20 +- .../actions/SegmentInsertActionTest.java | 4 +- .../SegmentTransactionalInsertActionTest.java | 6 +- .../task/batch/parallel/TaskMonitorTest.java | 2 +- .../SingleTaskBackgroundRunnerTest.java | 4 +- .../indexing/overlord/TaskLockboxTest.java | 109 +++++++---- .../indexing/overlord/http/OverlordTest.java | 4 +- .../OverlordSecurityResourceFilterTest.java | 2 +- ...nWithAffinityWorkerSelectStrategyTest.java | 6 +- ...lDistributionWorkerSelectStrategyTest.java | 10 +- ...yWithAffinityWorkerSelectStrategyTest.java | 6 +- .../worker/WorkerTaskManagerTest.java | 2 +- .../druid/java/util/common/IntRange.java | 91 --------- .../druid/java/util/common/IntRanges.java | 32 --- 18 files changed, 213 insertions(+), 290 deletions(-) delete mode 100644 java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java delete mode 100644 java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java 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 efb1c82da090..392ebbd24af0 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 @@ -29,6 +29,7 @@ @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 = SegmentLockAquireAction.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/task/NoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java index 02126093e2c7..470b9956ac30 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java @@ -74,6 +74,7 @@ enum IsReadyResult @JsonCreator public NoopTask( @JsonProperty("id") String id, + @JsonProperty("groupId") String groupId, @JsonProperty("dataSource") String dataSource, @JsonProperty("runTime") long runTime, @JsonProperty("isReadyTime") long isReadyTime, @@ -84,6 +85,8 @@ public NoopTask( { super( id == null ? StringUtils.format("noop_%s_%s", DateTimes.nowUtc(), UUID.randomUUID().toString()) : id, + groupId, + null, dataSource == null ? "none" : dataSource, context ); @@ -190,24 +193,29 @@ public int getPriority() public static NoopTask create() { - return new NoopTask(null, null, 0, 0, null, null, null); + return new NoopTask(null, null, null, 0, 0, null, null, null); + } + + public static NoopTask withGroupId(String groupId) + { + return new NoopTask(null, groupId, null, 0, 0, null, null, null); } @VisibleForTesting public static NoopTask create(String dataSource) { - return new NoopTask(null, dataSource, 0, 0, null, null, null); + return new NoopTask(null, null, dataSource, 0, 0, null, null, null); } @VisibleForTesting public static NoopTask create(int priority) { - return new NoopTask(null, null, 0, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, priority)); + return new NoopTask(null, null, null, 0, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, priority)); } @VisibleForTesting public static NoopTask create(String id, int priority) { - return new NoopTask(id, null, 0, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, priority)); + return new NoopTask(id, null, null, 0, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, priority)); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 0aa9247b7b9d..88e81675fc3e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -262,7 +262,7 @@ private TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLoc throw new ISE("Unknown lockGranularity[%s]", taskLock.getGranularity()); } - return createOrFindLockPosse(task, request); + return createOrFindLockPosse(request); } finally { giant.unlock(); @@ -371,7 +371,7 @@ public LockResult tryLock(final Task task, final LockRequest request) convertedRequest = request; } - final TaskLockPosse posseToUse = createOrFindLockPosse(task, convertedRequest); + final TaskLockPosse posseToUse = createOrFindLockPosse(convertedRequest); if (posseToUse != null && !posseToUse.getTaskLock().isRevoked()) { // Add to existing TaskLockPosse, if necessary if (posseToUse.addTask(task)) { @@ -412,14 +412,13 @@ public LockResult tryLock(final Task task, final LockRequest request) } } - private TaskLockPosse createOrFindLockPosse(Task task, LockRequest request) + private TaskLockPosse createOrFindLockPosse(LockRequest request) { Preconditions.checkState(!(request instanceof LockRequestForNewSegment), "Can't handle LockRequestForNewSegment"); giant.lock(); try { - final String taskId = task.getId(); final List foundPosses = findLockPossesOverlapsInterval( request.getDataSource(), request.getInterval() @@ -445,43 +444,35 @@ private TaskLockPosse createOrFindLockPosse(Task task, LockRequest request) // Any number of shared locks can be acquired for the same dataSource and interval. return createNewTaskLockPosse(request); } else { - if (isAllRevocable(conflictPosses, request.getPriority())) { - // Revoke all existing locks - conflictPosses.forEach(this::revokeLock); - + final boolean allDifferentGranularity = conflictPosses + .stream() + .allMatch( + conflictPosse -> conflictPosse.taskLock.getGranularity() != request.getGranularity() + && conflictPosse.getTaskLock().getGroupId().equals(request.getGroupId()) + && conflictPosse.getTaskLock().getInterval().equals(request.getInterval()) // TODO: contains? + ); + if (allDifferentGranularity) { + // We can add a new taskLockPosse return createNewTaskLockPosse(request); } else { - log.info( - "Cannot create a new taskLockPosse for request[%s] because existing locks[%s] have same or higher priorities", - request, - conflictPosses - ); - return null; + if (isAllRevocable(conflictPosses, request.getPriority())) { + // Revoke all existing locks + conflictPosses.forEach(this::revokeLock); + + return createNewTaskLockPosse(request); + } else { + log.info( + "Cannot create a new taskLockPosse for request[%s] because existing locks[%s] have same or higher priorities", + request, + conflictPosses + ); + return null; + } } } } else { // case 2) we found a lock posse for the given task - final TaskLockPosse foundPosse = reusablePosses.get(0); - if (request.getType().equals(foundPosse.getTaskLock().getLockType()) && - request.getGranularity() == foundPosse.getTaskLock().getGranularity()) { - return foundPosse; - } else { - if (request.getType() != foundPosse.getTaskLock().getLockType()) { - throw new ISE( - "Task[%s] already acquired a lock for interval[%s] but different type[%s]", - taskId, - request.getInterval(), - foundPosse.getTaskLock().getLockType() - ); - } else { - throw new ISE( - "Task[%s] already acquired a lock for interval[%s] but different granularity[%s]", - taskId, - request.getInterval(), - foundPosse.getTaskLock().getGranularity() - ); - } - } + return reusablePosses.get(0); } } else { // We don't have any locks for dataSource and interval. @@ -504,7 +495,6 @@ private TaskLockPosse createOrFindLockPosse(Task task, LockRequest request) * * @return a new {@link TaskLockPosse} */ - // TODO: new class for pair? private TaskLockPosse createNewTaskLockPosse(LockRequest request) { giant.lock(); @@ -584,7 +574,21 @@ private boolean isTaskLocksValid(Task task, Map> interva final boolean allLocksAreValid = lockPosses.stream().allMatch( posse -> !posse.getTaskLock().isRevoked() && posse.getTaskLock().getLockType() != TaskLockType.SHARED ); - return allLocksAreValid && lockPosses.size() == entry.getValue().size(); + + final Set remainingPartitionIds = new HashSet<>(entry.getValue()); + if (allLocksAreValid) { + for (TaskLockPosse lockPosse : lockPosses) { + if (lockPosse.getTaskLock().getGranularity() == LockGranularity.TIME_CHUNK) { + return true; + } else { + final SegmentLock segmentLock = (SegmentLock) lockPosse.getTaskLock(); + remainingPartitionIds.remove(segmentLock.getPartitionId()); + } + } + return remainingPartitionIds.isEmpty(); + } else { + return false; + } }); } finally { @@ -697,7 +701,9 @@ public void unlock(final Task task, final Interval interval, @Nullable Integer p try { final String dataSource = task.getDataSource(); - final NavigableMap>> dsRunning = running.get(task.getDataSource()); + final NavigableMap>> dsRunning = running.get( + task.getDataSource() + ); if (dsRunning == null || dsRunning.isEmpty()) { return; @@ -717,74 +723,59 @@ public void unlock(final Task task, final Interval interval, @Nullable Integer p final List posses = possesHolder.stream() .filter(posse -> posse.containsTask(task)) .collect(Collectors.toList()); + for (TaskLockPosse taskLockPosse : posses) { final TaskLock taskLock = taskLockPosse.getTaskLock(); - if (taskLock instanceof TimeChunkLock) { - if (partitionId != null) { - throw new ISE( - "PartitoinId[%s] are set, but lock granularity is timeChunk for task[%s] and interval[%s]", - partitionId, - task, - interval - ); - } - } else if (taskLock instanceof SegmentLock) { - if (partitionId == null) { - throw new ISE( - "PartitoinId are missing, but lock granularity is segmentLock for task[%s] and interval[%s]", - task, - interval - ); - } - - if (((SegmentLock) taskLock).getPartitionId() != partitionId) { - continue; - } - } + final boolean match = (partitionId == null && taskLock.getGranularity() == LockGranularity.TIME_CHUNK) + || (partitionId != null + && taskLock.getGranularity() == LockGranularity.SEGMENT + && ((SegmentLock) taskLock).getPartitionId() == partitionId); - // Remove task from live list - log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock); - final boolean removed = taskLockPosse.removeTask(task); + if (match) { + // Remove task from live list + log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock); + final boolean removed = taskLockPosse.removeTask(task); - if (taskLockPosse.isTasksEmpty()) { - log.info("TaskLock is now empty: %s", taskLock); - possesHolder.remove(taskLockPosse); - } + if (taskLockPosse.isTasksEmpty()) { + log.info("TaskLock is now empty: %s", taskLock); + possesHolder.remove(taskLockPosse); + } - if (possesHolder.isEmpty()) { - intervalToPosses.remove(interval); - } + if (possesHolder.isEmpty()) { + intervalToPosses.remove(interval); + } - if (intervalToPosses.isEmpty()) { - dsRunning.remove(interval.getStart()); - } + if (intervalToPosses.isEmpty()) { + dsRunning.remove(interval.getStart()); + } - if (running.get(dataSource).size() == 0) { - running.remove(dataSource); - } + if (running.get(dataSource).size() == 0) { + running.remove(dataSource); + } - // Wake up blocking-lock waiters - lockReleaseCondition.signalAll(); + // Wake up blocking-lock waiters + lockReleaseCondition.signalAll(); - // Remove lock from storage. If it cannot be removed, just ignore the failure. - try { - taskStorage.removeLock(task.getId(), taskLock); - } - catch (Exception e) { - log.makeAlert(e, "Failed to clean up lock from storage") - .addData("task", task.getId()) - .addData("dataSource", taskLock.getDataSource()) - .addData("interval", taskLock.getInterval()) - .addData("version", taskLock.getVersion()) - .emit(); - } + // Remove lock from storage. If it cannot be removed, just ignore the failure. + try { + taskStorage.removeLock(task.getId(), taskLock); + } + catch (Exception e) { + log.makeAlert(e, "Failed to clean up lock from storage") + .addData("task", task.getId()) + .addData("dataSource", taskLock.getDataSource()) + .addData("interval", taskLock.getInterval()) + .addData("version", taskLock.getVersion()) + .emit(); + } - if (!removed) { - log.makeAlert("Lock release without acquire") - .addData("task", task.getId()) - .addData("interval", interval) - .emit(); + if (!removed) { + log.makeAlert("Lock release without acquire") + .addData("task", task.getId()) + .addData("interval", interval) + .emit(); + } } } } @@ -1003,6 +994,9 @@ private static boolean isRevocable(TaskLockPosse lockPosse, int tryLockPriority) return existingLock.isRevoked() || existingLock.getNonNullPriority() < tryLockPriority; } + /** + * Task locks for tasks of the same groupId + */ static class TaskLockPosse { private final TaskLock taskLock; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java index 1809dd15216f..9ad3c23f6b4f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RemoteTaskActionClientTest.java @@ -95,7 +95,7 @@ public void testSubmitSimple() throws Exception expect(druidLeaderClient.go(request)).andReturn(responseHolder); replay(druidLeaderClient); - Task task = new NoopTask("id", null, 0, 0, null, null, null); + Task task = NoopTask.create("id", 0); RemoteTaskActionClient client = new RemoteTaskActionClient( task, druidLeaderClient, @@ -127,7 +127,7 @@ public void testSubmitWithIllegalStatusCode() throws Exception expect(druidLeaderClient.go(request)).andReturn(responseHolder); replay(druidLeaderClient); - Task task = new NoopTask("id", null, 0, 0, null, null, null); + Task task = NoopTask.create("id", 0); RemoteTaskActionClient client = new RemoteTaskActionClient( task, druidLeaderClient, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 0fcf1b5af04e..ba1fc2bf24fe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -118,7 +118,7 @@ public void testGranularitiesFinerThanHour() @Test public void testManySegmentsSameInterval() { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getTaskLockbox().add(task); @@ -187,7 +187,7 @@ public void testManySegmentsSameInterval() @Test public void testResumeSequence() { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getTaskLockbox().add(task); @@ -324,7 +324,7 @@ public void testResumeSequence() @Test public void testMultipleSequences() { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getTaskLockbox().add(task); @@ -433,7 +433,7 @@ public void testMultipleSequences() @Test public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( ImmutableSet.of( @@ -496,7 +496,7 @@ public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception @Test public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( ImmutableSet.of( @@ -557,7 +557,7 @@ public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exceptio @Test public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( ImmutableSet.of( @@ -594,7 +594,7 @@ public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() thr @Test public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( ImmutableSet.of( @@ -631,7 +631,7 @@ public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throw @Test public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( ImmutableSet.of( @@ -660,7 +660,7 @@ public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity @Test public void testCannotDoAnythingWithSillyQueryGranularity() { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getTaskLockbox().add(task); final SegmentIdWithShardSpec id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.HOUR, "s1", null); @@ -671,7 +671,7 @@ public void testCannotDoAnythingWithSillyQueryGranularity() @Test public void testCannotAddToExistingSingleDimensionShardSpecs() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); taskActionTestKit.getMetadataStorageCoordinator().announceHistoricalSegments( ImmutableSet.of( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java index c1b44c71d161..f8f04663826d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -99,7 +99,7 @@ private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interv @Test public void testSimple() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT1, SEGMENT2)); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); @@ -129,7 +129,7 @@ public void testSimple() throws Exception @Test public void testFailBadVersion() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); final SegmentInsertAction action = new SegmentInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java index cfae5f65024b..ed08c4fcb82c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java @@ -97,7 +97,7 @@ private LockResult acquireTimeChunkLock(TaskLockType lockType, Task task, Interv @Test public void testTransactional() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); @@ -138,7 +138,7 @@ public void testTransactional() throws Exception @Test public void testFailTransactional() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); @@ -157,7 +157,7 @@ public void testFailTransactional() throws Exception @Test public void testFailBadVersion() throws Exception { - final Task task = new NoopTask(null, null, 0, 0, null, null, null); + final Task task = NoopTask.create(); final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(ImmutableSet.of(SEGMENT3)); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java index 7e86e4072fc9..e5f85cb5dc99 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitorTest.java @@ -162,7 +162,7 @@ private static class TestTask extends NoopTask TestTask(String id, long runTime, boolean shouldFail) { - super(id, "testDataSource", runTime, 0, null, null, null); + super(id, null, "testDataSource", runTime, 0, null, null, null); this.shouldFail = shouldFail; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index c17b09d4883f..779baf9c2678 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -131,7 +131,7 @@ public void testRun() throws ExecutionException, InterruptedException { Assert.assertEquals( TaskState.SUCCESS, - runner.run(new NoopTask(null, null, 500L, 0, null, null, null)).get().getStatusCode() + runner.run(new NoopTask(null, null, null, 500L, 0, null, null, null)).get().getStatusCode() ); } @@ -139,7 +139,7 @@ public void testRun() throws ExecutionException, InterruptedException public void testStop() throws ExecutionException, InterruptedException, TimeoutException { final ListenableFuture future = runner.run( - new NoopTask(null, null, Long.MAX_VALUE, 0, null, null, null) // infinite task + new NoopTask(null, null, null, Long.MAX_VALUE, 0, null, null, null) // infinite task ); runner.stop(); Assert.assertEquals( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index a86fc1f69361..975aab01603a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -27,6 +27,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.Iterables; import org.apache.druid.indexer.TaskStatus; +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.TaskLockType; @@ -39,6 +40,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.TaskLockbox.TaskLockPosse; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -55,8 +57,10 @@ 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.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpecFactory; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -834,38 +838,6 @@ public void testSegmentAndTimeChunkLockForSameIntervalWithDifferentPriority() th Assert.assertTrue(resultOfTask1.isRevoked()); } - @Test - public void testLockWithDifferentGranularity() - { - final Task task = NoopTask.create("test", 10); - lockbox.add(task); - - Assert.assertTrue( - lockbox.tryLock( - task, - new TimeChunkLockRequest( - TaskLockType.EXCLUSIVE, - task, - Intervals.of("2015-01-01/2015-01-02"), - "v1" - ) - ).isOk() - ); - - Assert.assertFalse( - lockbox.tryLock( - task, - new SpecificSegmentLockRequest( - TaskLockType.EXCLUSIVE, - task, - Intervals.of("2015-01-01/2015-01-02"), - "v1", - 3 - ) - ).isOk() - ); - } - @Test public void testSegmentLockForSameIntervalAndSamePartition() { @@ -980,7 +952,7 @@ public void testRequestForNewSegmentWithSegmentLock() final Task task = NoopTask.create(); lockbox.add(task); allocateSegmentsAndAssert(task, "seq", 3, NumberedShardSpecFactory.instance()); - allocateSegmentsAndAssert(task, "seq2", 2, NumberedShardSpecFactory.instance()); //TODO ImmutableSet.of(0, 1, 2) for overwriting + allocateSegmentsAndAssert(task, "seq2", 2, new NumberedOverwritingShardSpecFactory(0, 3, (short) 1)); final List locks = lockbox.findLocksForTask(task); Assert.assertEquals(5, locks.size()); @@ -989,6 +961,9 @@ public void testRequestForNewSegmentWithSegmentLock() Assert.assertTrue(lock instanceof SegmentLock); final SegmentLock segmentLock = (SegmentLock) lock; Assert.assertEquals(expectedPartitionId++, segmentLock.getPartitionId()); + if (expectedPartitionId == 3) { + expectedPartitionId = ShardSpec.NON_ROOT_GEN_START_PARTITION_ID; + } } } @@ -1079,6 +1054,74 @@ public void testLockPosseEquals() Assert.assertEquals(taskLockPosse1, taskLockPosse3); } + @Test + public void testGetTimeChunkAndSegmentLockForSameGroup() + { + final Task task1 = NoopTask.withGroupId("groupId"); + final Task task2 = NoopTask.withGroupId("groupId"); + + lockbox.add(task1); + lockbox.add(task2); + + Assert.assertTrue( + lockbox.tryLock( + task1, + new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task1, Intervals.of("2017/2018"), null) + ).isOk() + ); + + Assert.assertTrue( + lockbox.tryLock( + task2, + new SpecificSegmentLockRequest(TaskLockType.EXCLUSIVE, task2, Intervals.of("2017/2018"), "version", 0) + ).isOk() + ); + + final List posses = lockbox + .getAllLocks() + .get(task1.getDataSource()) + .get(DateTimes.of("2017")) + .get(Intervals.of("2017/2018")); + Assert.assertEquals(2, posses.size()); + + Assert.assertEquals(LockGranularity.TIME_CHUNK, posses.get(0).getTaskLock().getGranularity()); + final TimeChunkLock timeChunkLock = (TimeChunkLock) posses.get(0).getTaskLock(); + Assert.assertEquals("none", timeChunkLock.getDataSource()); + Assert.assertEquals("groupId", timeChunkLock.getGroupId()); + Assert.assertEquals(Intervals.of("2017/2018"), timeChunkLock.getInterval()); + + Assert.assertEquals(LockGranularity.SEGMENT, posses.get(1).getTaskLock().getGranularity()); + final SegmentLock segmentLock = (SegmentLock) posses.get(1).getTaskLock(); + Assert.assertEquals("none", segmentLock.getDataSource()); + Assert.assertEquals("groupId", segmentLock.getGroupId()); + Assert.assertEquals(Intervals.of("2017/2018"), segmentLock.getInterval()); + Assert.assertEquals(0, segmentLock.getPartitionId()); + } + + @Test + public void testGetTimeChunkAndSegmentLockForDifferentGroup() + { + final Task task1 = NoopTask.withGroupId("groupId"); + final Task task2 = NoopTask.withGroupId("groupId2"); + + lockbox.add(task1); + lockbox.add(task2); + + Assert.assertTrue( + lockbox.tryLock( + task1, + new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task1, Intervals.of("2017/2018"), null) + ).isOk() + ); + + Assert.assertFalse( + lockbox.tryLock( + task2, + new SpecificSegmentLockRequest(TaskLockType.EXCLUSIVE, task2, Intervals.of("2017/2018"), "version", 0) + ).isOk() + ); + } + private Set getAllLocks(List tasks) { return tasks.stream() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index ecf2af43524e..a45bef451558 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -226,7 +226,7 @@ public void testOverlordRun() throws Exception Assert.assertEquals(druidNode.getHostAndPort(), response.getEntity()); final String taskId_0 = "0"; - NoopTask task_0 = new NoopTask(taskId_0, null, 0, 0, null, null, null); + NoopTask task_0 = NoopTask.create(taskId_0, 0); response = overlordResource.taskPost(task_0, req); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(ImmutableMap.of("task", taskId_0), response.getEntity()); @@ -259,7 +259,7 @@ public void testOverlordRun() throws Exception // Manually insert task in taskStorage // Verifies sync from storage final String taskId_1 = "1"; - NoopTask task_1 = new NoopTask(taskId_1, null, 0, 0, null, null, null); + NoopTask task_1 = NoopTask.create(taskId_1, 0); taskStorage.insert(task_1, TaskStatus.running(taskId_1)); // Wait for task runner to run task_1 runTaskCountDownLatches[Integer.parseInt(taskId_1)].await(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java index 16c8afea1851..7977164b1ca6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java @@ -78,7 +78,7 @@ public static Collection data() private final String requestMethod; private final ResourceFilter resourceFilter; private final Injector injector; - private final Task noopTask = new NoopTask(null, null, 0, 0, null, null, null); + private final Task noopTask = NoopTask.create(); private static boolean mockedOnceTsqa; private static boolean mockedOnceSM; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java index 29294b8ca20c..3c0bddbce950 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithAffinityWorkerSelectStrategyTest.java @@ -42,7 +42,7 @@ public void testFindWorkerForTask() new AffinityConfig(ImmutableMap.of("foo", ImmutableSet.of("localhost1", "localhost2", "localhost3")), false) ); - NoopTask noopTask = new NoopTask(null, null, 1, 0, null, null, null) + NoopTask noopTask = new NoopTask(null, null, null, 1, 0, null, null, null) { @Override public String getDataSource() @@ -112,7 +112,7 @@ public void testFindWorkerForTaskWithNulls() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) ); Assert.assertEquals("lhost", worker.getWorker().getHost()); } @@ -135,7 +135,7 @@ public void testIsolation() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) ); Assert.assertNull(worker); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java index e68d0a9312a6..111b8df990c0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWorkerSelectStrategyTest.java @@ -88,7 +88,7 @@ public void testFindWorkerForTask() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) { @Override public String getDataSource() @@ -123,7 +123,7 @@ public void testFindWorkerForTaskWhenSameCurrCapacityUsed() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) { @Override public String getDataSource() @@ -159,7 +159,7 @@ public void testOneDisableWorkerDifferentUsedCapacity() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) { @Override public String getDataSource() @@ -195,7 +195,7 @@ public void testOneDisableWorkerSameUsedCapacity() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) { @Override public String getDataSource() @@ -281,7 +281,7 @@ public void testStrongAffinity() private static NoopTask createDummyTask(final String dataSource) { - return new NoopTask(null, null, 1, 0, null, null, null) + return new NoopTask(null, null, null, 1, 0, null, null, null) { @Override public String getDataSource() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithAffinityWorkerSelectStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithAffinityWorkerSelectStrategyTest.java index 73599f502ed6..dc2cba48ad95 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithAffinityWorkerSelectStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithAffinityWorkerSelectStrategyTest.java @@ -58,7 +58,7 @@ public void testFindWorkerForTask() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) { @Override public String getDataSource() @@ -95,7 +95,7 @@ public void testFindWorkerForTaskWithNulls() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) ); Assert.assertEquals("lhost", worker.getWorker().getHost()); } @@ -118,7 +118,7 @@ public void testIsolation() DateTimes.nowUtc() ) ), - new NoopTask(null, null, 1, 0, null, null, null) + new NoopTask(null, null, null, 1, 0, null, null, null) ); Assert.assertNull(worker); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index b86b654847a4..b3821a29f594 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -258,6 +258,6 @@ public void testTaskRun() throws Exception private NoopTask createNoopTask(String id) { - return new NoopTask(id, null, 100, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, 0)); + return new NoopTask(id, null, null, 100, 0, null, null, ImmutableMap.of(Tasks.PRIORITY_KEY, 0)); } } diff --git a/java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java b/java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java deleted file mode 100644 index 1b37175e8e57..000000000000 --- a/java-util/src/main/java/org/apache/druid/java/util/common/IntRange.java +++ /dev/null @@ -1,91 +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.java.util.common; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.Objects; - -public class IntRange -{ - private final int start; - private final int len; - - @JsonCreator - public IntRange( - @JsonProperty("start") int start, - @JsonProperty("len") int len - ) - { - this.start = start; - this.len = len; - } - - public boolean contains(int i) - { - return i >= start && i < getExclusiveEnd(); - } - - @JsonProperty - public int getStart() - { - return start; - } - - @JsonProperty - public int getLen() - { - return len; - } - - public int getExclusiveEnd() - { - return start + len; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - IntRange intRange = (IntRange) o; - return start == intRange.start && - len == intRange.len; - } - - @Override - public int hashCode() - { - return Objects.hash(start, len); - } - - @Override - public String toString() - { - return "IntRange{" + - "start=" + start + - ", len=" + len + - '}'; - } -} diff --git a/java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java b/java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java deleted file mode 100644 index 534ab3290b37..000000000000 --- a/java-util/src/main/java/org/apache/druid/java/util/common/IntRanges.java +++ /dev/null @@ -1,32 +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.java.util.common; - -import java.util.Comparator; - -public final class IntRanges -{ - public Comparator startThenEnd() - { - return Comparator.comparingInt(IntRange::getStart) - .thenComparing(range -> range.getStart() + range.getLen()); - } - - private IntRanges() {} -} From e47dcdc35b617a06db0ae18c74263b9f83839f80 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Apr 2019 13:03:16 -0700 Subject: [PATCH 03/52] fix it test --- .../CoordinatorResourceTestClient.java | 8 ++----- .../indexer/AbstractITBatchIndexTest.java | 21 ++++++++++++++++--- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java index babb9e3ead3d..23f5956e8594 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java @@ -45,8 +45,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; public class CoordinatorResourceTestClient { @@ -135,13 +133,12 @@ public List getSegmentIntervals(final String dataSource) } // return a set of the segment versions for the specified datasource - public Set getSegmentVersions(final String dataSource) + public List getAvailableSegments(final String dataSource) { - ArrayList segments; try { StatusResponseHolder response = makeRequest(HttpMethod.GET, getFullSegmentsURL(dataSource)); - segments = jsonMapper.readValue( + return jsonMapper.readValue( response.getContent(), new TypeReference>() { } @@ -150,7 +147,6 @@ public Set getSegmentVersions(final String dataSource) catch (Exception e) { throw new RuntimeException(e); } - return segments.stream().map(s -> s.getVersion()).collect(Collectors.toSet()); } private Map getLoadStatus() diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index 0c6a64eb0aa6..cb3891a1f0af 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -22,18 +22,21 @@ import com.google.inject.Inject; import org.apache.commons.io.IOUtils; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.clients.ClientInfoResourceTestClient; import org.apache.druid.testing.utils.RetryUtil; import org.apache.druid.testing.utils.SqlTestQueryHelper; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.VersionedIntervalTimeline; import org.junit.Assert; import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.Set; public class AbstractITBatchIndexTest extends AbstractIndexerTest { @@ -165,7 +168,7 @@ void doIndexTestSqlTest( private void submitTaskAndWait(String taskSpec, String dataSourceName, boolean waitForNewVersion) { - final Set oldVersions = waitForNewVersion ? coordinator.getSegmentVersions(dataSourceName) : null; + final List oldVersions = waitForNewVersion ? coordinator.getAvailableSegments(dataSourceName) : null; long startSubTaskCount = -1; final boolean assertRunsSubTasks = taskSpec.contains("index_parallel"); @@ -193,7 +196,19 @@ private void submitTaskAndWait(String taskSpec, String dataSourceName, boolean w // original segments have loaded. if (waitForNewVersion) { RetryUtil.retryUntilTrue( - () -> !oldVersions.containsAll(coordinator.getSegmentVersions(dataSourceName)), "See a new version" + () -> { + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + coordinator.getAvailableSegments(dataSourceName) + ); + + final List> holders = timeline.lookup(Intervals.ETERNITY); + return holders + .stream() + .flatMap(holder -> holder.getObject().stream()) + .anyMatch(chunk -> oldVersions.stream() + .anyMatch(oldSegment -> chunk.getObject().isOvershadow(oldSegment))); + }, + "See a new version" ); } From 16b56cf97c51c7eeef4bb6388e952ac7921bbd1b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Apr 2019 15:18:56 -0700 Subject: [PATCH 04/52] Fix adding same chunk to atomicUpdateGroup --- .../timeline/partition/AtomicUpdateGroup.java | 15 +++--- .../partition/OvershadowableManager.java | 2 + .../VersionedIntervalTimelineTest.java | 50 +++++++++++++++++++ 3 files changed, 60 insertions(+), 7 deletions(-) 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 index 9311b9846076..1c6db6a0ffcd 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -41,13 +41,14 @@ public AtomicUpdateGroup(PartitionChunk chunk) public void add(PartitionChunk chunk) { - if (isFull()) { - throw new IAE("Can't add more chunk[%s] to atomicUpdateGroup[%s]", chunk, chunks); - } - if (!isSameAtomicUpdateGroup(chunks.get(0), chunk)) { - throw new IAE("Can't add chunk[%s] to a different atomicUpdateGroup[%s]", chunk, chunks); - } - if (replaceChunkWith(chunk) == null) { + final PartitionChunk existing = replaceChunkWith(chunk); + if (existing == null) { + if (isFull()) { + throw new IAE("Can't add more chunk[%s] to atomicUpdateGroup[%s]", chunk, chunks); + } + if (!isSameAtomicUpdateGroup(chunks.get(0), chunk)) { + throw new IAE("Can't add chunk[%s] to a different atomicUpdateGroup[%s]", chunk, chunks); + } chunks.add(chunk); } } 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 index f7e5975781aa..91c31217977f 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -196,6 +196,8 @@ private void handleAdd(AtomicUpdateGroup aug, State newStateOfAug) // visible which overshadows the current visible one. findOvershadowedBy(aug, State.VISIBLE) .forEach(entry -> transitPartitionChunkState(entry.getValue(), State.VISIBLE, State.OVERSHADOWED)); + findOvershadowedBy(aug, State.STANDBY) + .forEach(entry -> transitPartitionChunkState(entry.getValue(), State.STANDBY, State.OVERSHADOWED)); transitPartitionChunkState(aug, State.STANDBY, State.VISIBLE); } } 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 986e36aaca1f..f73d26ec8919 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -1912,6 +1912,56 @@ public void testFallbackOnMissingSegment() ); } + @Test + public void testAddSameChunkToFullAtomicUpdateGroup() + { + timeline = makeStringIntegerTimeline(); + final Interval interval = Intervals.of("2019-01-01/2019-01-02"); + add(interval, "0", makeNumbered(0, 0)); + add(interval, "0", makeNumberedOverwriting(0, 0, 0, 1, 1, 1)); + add(interval, "0", makeNumbered(0, 1)); + + final Set> overshadowed = timeline.findFullyOvershadowed(); + Assert.assertEquals( + ImmutableSet.of( + new TimelineObjectHolder<>( + interval, + "0", + new PartitionHolder<>(ImmutableList.of(makeNumbered(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, 1, 1, 2)); + add(interval, "0", makeNumberedOverwriting(1, 0, 0, 1, 2, 2)); + add(interval, "0", makeNumberedOverwriting(2, 0, 0, 1, 3, 2)); // <-- full atomicUpdateGroup + add(interval, "0", makeNumberedOverwriting(3, 1, 0, 1, 3, 2)); // <-- full atomicUpdateGroup + + final Set> overshadowed = timeline.findFullyOvershadowed(); + Assert.assertEquals( + ImmutableSet.of( + new TimelineObjectHolder<>( + interval, + "0", + new PartitionHolder<>( + ImmutableList.of( + makeNumberedOverwriting(0, 0, 0, 1, 1, 2), + makeNumberedOverwriting(1, 0, 0, 1, 2, 2) + ) + ) + ) + ), + overshadowed + ); + } + // TODO: test if the middle segment is missing (B among A <- B <- C) // TODO: test fall back when the middle segment is missing From a915a63806a987cbdf4ff816b91a752eed44f115 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Apr 2019 17:22:01 -0700 Subject: [PATCH 05/52] resolving todos --- .../timeline/VersionedIntervalTimeline.java | 2 -- .../timeline/partition/AtomicUpdateGroup.java | 6 ++++- .../partition/HashBasedNumberedShardSpec.java | 4 ---- .../NumberedOverwritingShardSpec.java | 1 - .../partition/OvershadowableManager.java | 4 +--- .../VersionedIntervalTimelineTest.java | 22 ++++++++++++++++--- 6 files changed, 25 insertions(+), 14 deletions(-) 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 0762fa78c78d..551b8bbc9b28 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -374,8 +374,6 @@ public Set> findFullyOvershadowed( } } - // TODO: check this is correct - // TODO: add test public boolean isOvershadowed(Interval interval, VersionType version, ObjectType object) { try { 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 index 1c6db6a0ffcd..fe76daba1f97 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -31,7 +31,11 @@ class AtomicUpdateGroup> implements Overshadowable> { - // TODO: map?? + // 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> chunks = new ArrayList<>(); public AtomicUpdateGroup(PartitionChunk chunk) 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 760558958cd2..88dfc8397d25 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 @@ -45,10 +45,6 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec @JsonIgnore private final List partitionDimensions; - // TODO: check what's the valid partitions. is it (max existing partitionId) + # of new partitions? - // or just # of new partitions? - // or do we need a start partitionId? - @JsonCreator public HashBasedNumberedShardSpec( @JsonProperty("partitionNum") int partitionNum, // partitionId diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java index 9f7857fa57ed..bbf4e5bc012a 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java @@ -30,7 +30,6 @@ import java.util.Map; import java.util.Objects; -// TODO: abstract numbered?? public class NumberedOverwritingShardSpec implements OverwritingShardSpec { private final int partitionId; 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 index 91c31217977f..634c8f5da4f2 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -39,11 +39,9 @@ import java.util.TreeMap; import java.util.stream.Collectors; -// TODO: rename - /** + * * Not thread-safe - * @param */ public class OvershadowableManager> { 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 f73d26ec8919..2ad208114186 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -1962,11 +1962,27 @@ public void testOvershadowMultipleStandbyAtomicUpdateGroup() ); } - // TODO: test if the middle segment is missing (B among A <- B <- C) + @Test + public void testIsOvershadowedForOverwritingSegments() + { + timeline = makeStringIntegerTimeline(); + final Interval interval = Intervals.of("2019-01-01/2019-01-02"); + add(interval, "0", makeNumberedOverwriting(0, 0, 5, 10, 10, 1)); - // TODO: test fall back when the middle segment is missing + for (int i = 0; i < 5; i++) { + Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumbered(i + 5, 0).getObject())); + } + + Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumbered(4, 0).getObject())); + Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumbered(11, 0).getObject())); - // TODO: test isOvershadowed() + Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 5, 6, 5, 2).getObject())); + Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 7, 8, 5, 2).getObject())); + Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 8, 10, 5, 2).getObject())); + + Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 5, 10, 12, 2).getObject())); + Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 4, 15, 12, 2).getObject())); + } private TimelineObjectHolder makeTimelineObjectHolder( String interval, From 30dc9b93c366f406b19db5e84dbb6322b9e851b4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Apr 2019 20:56:55 -0700 Subject: [PATCH 06/52] Fix segments to lock --- .../task/AbstractFixedIntervalTask.java | 2 +- .../indexing/common/task/AbstractTask.java | 59 ++++++++++++------- .../indexing/common/task/CompactionTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 2 +- .../druid/indexing/common/task/KillTask.java | 10 +--- .../batch/parallel/ParallelIndexSubTask.java | 4 +- .../parallel/ParallelIndexSupervisorTask.java | 2 +- 7 files changed, 46 insertions(+), 35 deletions(-) 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 90abc3b6b79e..373795b97366 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 @@ -71,7 +71,7 @@ protected AbstractFixedIntervalTask( @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - return tryLockWithIntervals(taskActionClient, Collections.singletonList(interval)); + return tryLockWithIntervals(taskActionClient, Collections.singletonList(interval), false); } @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 a73eb254cfa6..43abffe6d9cd 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 @@ -41,7 +41,6 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineLookup; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; @@ -79,7 +78,7 @@ public abstract class AbstractTask implements Task private final Map context; @Nullable - private Map overwritingSegmentMetas; + private Map overwritingSegmentMetas; // TODO: rename? @Nullable private Boolean changeSegmentGranularity; @@ -314,13 +313,13 @@ public Map getContext() @Nullable public abstract Granularity getSegmentGranularity(Interval interval); - protected boolean tryLockWithIntervals(TaskActionClient client, Set intervals) + boolean tryLockWithIntervals(TaskActionClient client, Set intervals) throws IOException { - return tryLockWithIntervals(client, new ArrayList<>(intervals)); + return tryLockWithIntervals(client, new ArrayList<>(intervals), true); } - protected boolean tryLockWithIntervals(TaskActionClient client, List intervals) + protected boolean tryLockWithIntervals(TaskActionClient client, List intervals, boolean lockVisibleSegments) throws IOException { if (requireLockInputSegments()) { @@ -333,7 +332,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i // TODO: race - a new segment can be added after findInputSegments. change to lockAllSegmentsInIntervals if (!intervalsToFindInput.isEmpty()) { - return tryLockWithSegments(client, findInputSegments(client, intervalsToFindInput)); + return tryLockWithSegments(client, findInputSegments(client, intervalsToFindInput), lockVisibleSegments); } else { return true; } @@ -344,7 +343,8 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } - protected boolean tryLockWithSegments(TaskActionClient client, List segments) throws IOException + boolean tryLockWithSegments(TaskActionClient client, List segments, boolean lockVisibleSegments) + throws IOException { if (requireLockInputSegments()) { if (segments.isEmpty()) { @@ -355,16 +355,6 @@ protected boolean tryLockWithSegments(TaskActionClient client, List // Create a timeline to find latest segments only final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); - final TimelineLookup timeline = VersionedIntervalTimeline.forSegments(segments); - final List visibleSegments = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) - .stream() - .map(TimelineObjectHolder::getObject) - .flatMap(partitionHolder -> StreamSupport.stream( - partitionHolder.spliterator(), - false - )) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); changeSegmentGranularity = changeSegmentGranularity(intervals); if (changeSegmentGranularity) { @@ -388,20 +378,43 @@ protected boolean tryLockWithSegments(TaskActionClient client, List } return true; } else { + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(segments); + final List segmentsToLock; + if (lockVisibleSegments) { + segmentsToLock = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) + .stream() + .map(TimelineObjectHolder::getObject) + .flatMap(partitionHolder -> StreamSupport.stream( + partitionHolder.spliterator(), + false + )) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); + } else { + segmentsToLock = timeline.findFullyOvershadowed() + .stream() + .flatMap(holder -> holder.getObject().stream()) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); + } + final Map> intervalToSegments = new HashMap<>(); - for (DataSegment segment : segments) { + for (DataSegment segment : segmentsToLock) { intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); } - intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); + if (lockVisibleSegments) { + intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); + } for (Entry> entry : intervalToSegments.entrySet()) { final Interval interval = entry.getKey(); final Set partitionIds = entry.getValue().stream() .map(s -> s.getShardSpec().getPartitionNum()) .collect(Collectors.toSet()); final List lockResults = client.submit( - new SegmentLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval, visibleSegments.get(0).getVersion(), partitionIds) + new SegmentLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval, entry.getValue().get(0).getVersion(), partitionIds) ); if (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk())) { + // TODO: unlock return false; } } @@ -414,6 +427,10 @@ protected boolean tryLockWithSegments(TaskActionClient client, List } } + /** + * TODO: description about what this method verifies + * TODO: similar check in taskLockbox?? + */ private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments) { if (inputSegments.isEmpty()) { @@ -485,7 +502,7 @@ protected boolean isChangeSegmentGranularity() return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); } - public Map getAllOverwritingSegmentMeta() + Map getAllOverwritingSegmentMeta() { Preconditions.checkNotNull(overwritingSegmentMetas, "overwritingSegmentMetas is not initialized"); return overwritingSegmentMetas; 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 b93104769fe1..58f3a05c5b7c 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 @@ -281,7 +281,7 @@ SegmentProvider getSegmentProvider() public boolean isReady(TaskActionClient taskActionClient) throws Exception { final List segments = segmentProvider.checkAndGetSegments(taskActionClient); - return tryLockWithSegments(taskActionClient, segments); + return tryLockWithSegments(taskActionClient, segments, true); } @Override 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 3cf0f6706b4d..c2020a87b73b 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 @@ -296,7 +296,7 @@ private boolean tryLockIfNecessary(TaskActionClient actionClient, Collection(intervals)); + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals), true); } @GET diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java index b2da38364d11..2e20d59b5a4c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableSet; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; @@ -31,7 +30,6 @@ import org.apache.druid.indexing.common.actions.TaskActionPreconditions; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -39,6 +37,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -49,8 +48,6 @@ */ public class KillTask extends AbstractFixedIntervalTask { - private static final Logger log = new Logger(KillTask.class); - @JsonCreator public KillTask( @JsonProperty("id") String id, @@ -112,9 +109,6 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception .getTaskActionClient() .submit(new SegmentListUnusedAction(getDataSource(), getInterval())); - // log.info("segments to kill: %s", unusedSegments); - // log.info("taskLockMap: %s", taskLockMap); - if (!TaskActionPreconditions.isLockCoversSegments(taskLockMap, unusedSegments)) { throw new ISE( "Locks[%s] for task[%s] can't cover segments[%s]", @@ -125,9 +119,9 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception } // Kill segments + toolbox.getTaskActionClient().submit(new SegmentNukeAction(new HashSet<>(unusedSegments))); for (DataSegment segment : unusedSegments) { toolbox.getDataSegmentKiller().kill(segment); - toolbox.getTaskActionClient().submit(new SegmentNukeAction(ImmutableSet.of(segment))); } return TaskStatus.success(getId()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 7b9189995639..54bd1e612567 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -158,7 +158,7 @@ public boolean isReady(TaskActionClient taskActionClient) private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) { try { - return tryLockWithIntervals(actionClient, new ArrayList<>(intervals)); + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals), true); } catch (Exception e) { log.error(e, "Failed to acquire locks for intervals[%s]", intervals); @@ -416,7 +416,7 @@ private List generateAndPushSegments( } else { final Granularity segmentGranularity = findSegmentGranularity(granularitySpec); final Interval timeChunk = segmentGranularity.bucket(inputRow.getTimestamp()); - if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk))) { + if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk), true)) { throw new ISE("Failed to get locks for interval[%s]", timeChunk); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 6285cea27cbf..60d37bf7a176 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -221,7 +221,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception .getGranularitySpec() .bucketIntervals(); - return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get())); + return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get()), true); } @Override From 3378f1a0b81b8aeb24fe7f3a5596addef36a8a16 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Apr 2019 21:50:32 -0700 Subject: [PATCH 07/52] fix segments to lock --- .../druid/indexing/common/task/AbstractTask.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) 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 43abffe6d9cd..84226f4a7fb0 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 @@ -343,7 +343,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } - boolean tryLockWithSegments(TaskActionClient client, List segments, boolean lockVisibleSegments) + boolean tryLockWithSegments(TaskActionClient client, List segments, boolean lockVisibleSegmentsOnly) throws IOException { if (requireLockInputSegments()) { @@ -378,9 +378,11 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, } return true; } else { - final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(segments); final List segmentsToLock; - if (lockVisibleSegments) { + if (lockVisibleSegmentsOnly) { + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + segments + ); segmentsToLock = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) .stream() .map(TimelineObjectHolder::getObject) @@ -391,18 +393,14 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, .map(PartitionChunk::getObject) .collect(Collectors.toList()); } else { - segmentsToLock = timeline.findFullyOvershadowed() - .stream() - .flatMap(holder -> holder.getObject().stream()) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); + segmentsToLock = segments; } final Map> intervalToSegments = new HashMap<>(); for (DataSegment segment : segmentsToLock) { intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); } - if (lockVisibleSegments) { + if (lockVisibleSegmentsOnly) { intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); } for (Entry> entry : intervalToSegments.entrySet()) { From 95b4e0994bdfa30ca7e7b4855ea8fd59b18c876f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 24 Apr 2019 00:43:25 -0700 Subject: [PATCH 08/52] fix kill task --- .../task/AbstractFixedIntervalTask.java | 5 ++- .../indexing/common/task/AbstractTask.java | 40 ++++++++----------- .../indexing/common/task/CompactionTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 2 +- .../batch/parallel/ParallelIndexSubTask.java | 4 +- .../parallel/ParallelIndexSupervisorTask.java | 2 +- .../indexing/common/task/KillTaskTest.java | 24 ++++++----- 7 files changed, 38 insertions(+), 41 deletions(-) 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 373795b97366..9582b987876c 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 @@ -22,10 +22,11 @@ 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.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.joda.time.Interval; -import java.util.Collections; import java.util.Map; public abstract class AbstractFixedIntervalTask extends AbstractTask @@ -71,7 +72,7 @@ protected AbstractFixedIntervalTask( @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - return tryLockWithIntervals(taskActionClient, Collections.singletonList(interval), false); + 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 84226f4a7fb0..40a13dac9e85 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 @@ -316,10 +316,10 @@ public Map getContext() boolean tryLockWithIntervals(TaskActionClient client, Set intervals) throws IOException { - return tryLockWithIntervals(client, new ArrayList<>(intervals), true); + return tryLockWithIntervals(client, new ArrayList<>(intervals)); } - protected boolean tryLockWithIntervals(TaskActionClient client, List intervals, boolean lockVisibleSegments) + protected boolean tryLockWithIntervals(TaskActionClient client, List intervals) throws IOException { if (requireLockInputSegments()) { @@ -332,7 +332,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i // TODO: race - a new segment can be added after findInputSegments. change to lockAllSegmentsInIntervals if (!intervalsToFindInput.isEmpty()) { - return tryLockWithSegments(client, findInputSegments(client, intervalsToFindInput), lockVisibleSegments); + return tryLockWithSegments(client, findInputSegments(client, intervalsToFindInput)); } else { return true; } @@ -343,7 +343,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } - boolean tryLockWithSegments(TaskActionClient client, List segments, boolean lockVisibleSegmentsOnly) + boolean tryLockWithSegments(TaskActionClient client, List segments) throws IOException { if (requireLockInputSegments()) { @@ -379,30 +379,24 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, return true; } else { final List segmentsToLock; - if (lockVisibleSegmentsOnly) { - final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( - segments - ); - segmentsToLock = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) - .stream() - .map(TimelineObjectHolder::getObject) - .flatMap(partitionHolder -> StreamSupport.stream( - partitionHolder.spliterator(), - false - )) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); - } else { - segmentsToLock = segments; - } + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + segments + ); + segmentsToLock = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) + .stream() + .map(TimelineObjectHolder::getObject) + .flatMap(partitionHolder -> StreamSupport.stream( + partitionHolder.spliterator(), + false + )) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); final Map> intervalToSegments = new HashMap<>(); for (DataSegment segment : segmentsToLock) { intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); } - if (lockVisibleSegmentsOnly) { - intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); - } + intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); for (Entry> entry : intervalToSegments.entrySet()) { final Interval interval = entry.getKey(); final Set partitionIds = entry.getValue().stream() 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 58f3a05c5b7c..b93104769fe1 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 @@ -281,7 +281,7 @@ SegmentProvider getSegmentProvider() public boolean isReady(TaskActionClient taskActionClient) throws Exception { final List segments = segmentProvider.checkAndGetSegments(taskActionClient); - return tryLockWithSegments(taskActionClient, segments, true); + return tryLockWithSegments(taskActionClient, segments); } @Override 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 c2020a87b73b..3cf0f6706b4d 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 @@ -296,7 +296,7 @@ private boolean tryLockIfNecessary(TaskActionClient actionClient, Collection(intervals), true); + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals)); } @GET diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 54bd1e612567..7b9189995639 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -158,7 +158,7 @@ public boolean isReady(TaskActionClient taskActionClient) private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) { try { - return tryLockWithIntervals(actionClient, new ArrayList<>(intervals), true); + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals)); } catch (Exception e) { log.error(e, "Failed to acquire locks for intervals[%s]", intervals); @@ -416,7 +416,7 @@ private List generateAndPushSegments( } else { final Granularity segmentGranularity = findSegmentGranularity(granularitySpec); final Interval timeChunk = segmentGranularity.bucket(inputRow.getTimestamp()); - if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk), true)) { + if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk))) { throw new ISE("Failed to get locks for interval[%s]", timeChunk); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 60d37bf7a176..6285cea27cbf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -221,7 +221,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception .getGranularitySpec() .bucketIntervals(); - return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get()), true); + return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get())); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java index 33bf5b35d3d7..1492a4749de7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -48,11 +49,12 @@ public void setup() @Test public void testKill() throws Exception { + final String version = DateTimes.nowUtc().toString(); final Set segments = ImmutableSet.of( - newSegment(Intervals.of("2019-01-01/2019-02-01")), - newSegment(Intervals.of("2019-02-01/2019-03-01")), - newSegment(Intervals.of("2019-03-01/2019-04-01")), - newSegment(Intervals.of("2019-04-01/2019-05-01")) + newSegment(Intervals.of("2019-01-01/2019-02-01"), version), + newSegment(Intervals.of("2019-02-01/2019-03-01"), version), + newSegment(Intervals.of("2019-03-01/2019-04-01"), version), + newSegment(Intervals.of("2019-04-01/2019-05-01"), version) ); final Set announced = getMetadataStorageCoordinator().announceHistoricalSegments(segments); @@ -61,13 +63,13 @@ public void testKill() throws Exception Assert.assertTrue( getMetadataSegmentManager().removeSegment( DATA_SOURCE, - newSegment(Intervals.of("2019-02-01/2019-03-01")).getId().toString() + newSegment(Intervals.of("2019-02-01/2019-03-01"), version).getId().toString() ) ); Assert.assertTrue( getMetadataSegmentManager().removeSegment( DATA_SOURCE, - newSegment(Intervals.of("2019-03-01/2019-04-01")).getId().toString() + newSegment(Intervals.of("2019-03-01/2019-04-01"), version).getId().toString() ) ); @@ -80,22 +82,22 @@ public void testKill() throws Exception Intervals.of("2019/2020") ); - Assert.assertEquals(ImmutableList.of(newSegment(Intervals.of("2019-02-01/2019-03-01"))), unusedSegments); + Assert.assertEquals(ImmutableList.of(newSegment(Intervals.of("2019-02-01/2019-03-01"), version)), unusedSegments); Assert.assertEquals( ImmutableList.of( - newSegment(Intervals.of("2019-01-01/2019-02-01")), - newSegment(Intervals.of("2019-04-01/2019-05-01")) + newSegment(Intervals.of("2019-01-01/2019-02-01"), version), + newSegment(Intervals.of("2019-04-01/2019-05-01"), version) ), getMetadataStorageCoordinator().getUsedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020")) ); } - private static DataSegment newSegment(Interval interval) + private static DataSegment newSegment(Interval interval, String version) { return new DataSegment( DATA_SOURCE, interval, - "version", + version, null, null, null, From 3f128e4bccd63c4a623d092cb18ec60fe80bffc8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 24 Apr 2019 13:19:29 -0700 Subject: [PATCH 09/52] resolving todos --- .../NumberedOverwritingPartitionChunk.java | 8 +- .../NumberedOverwritingShardSpec.java | 27 ++- .../NumberedOverwritingShardSpecFactory.java | 2 +- .../partition/OvershadowableManager.java | 206 +++++++++++++++++- ...gPartitionChunk.java => PartitionIds.java} | 10 +- .../druid/timeline/partition/ShardSpec.java | 7 - .../VersionedIntervalTimelineTest.java | 4 +- .../druid/indexing/common/SegmentLock.java | 3 +- .../common/actions/SegmentAllocateAction.java | 8 +- .../actions/SegmentLockTryAcquireAction.java | 2 +- .../indexing/common/task/AbstractTask.java | 73 ++++--- .../druid/indexing/common/task/IndexTask.java | 11 +- .../common/task/RemoteSegmentAllocator.java | 6 +- .../batch/parallel/ParallelIndexSubTask.java | 2 +- .../common/task/CompactionTaskRunTest.java | 37 +++- .../indexing/overlord/TaskLockboxTest.java | 4 +- ...exerSQLMetadataStorageCoordinatorTest.java | 6 +- 17 files changed, 325 insertions(+), 91 deletions(-) rename core/src/main/java/org/apache/druid/timeline/partition/{OverwritingPartitionChunk.java => PartitionIds.java} (69%) 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 index dbe1b1060ac5..cc355092befc 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java @@ -24,7 +24,7 @@ import java.util.Objects; -public class NumberedOverwritingPartitionChunk implements OverwritingPartitionChunk +public class NumberedOverwritingPartitionChunk implements PartitionChunk { private final int chunkId; private final T object; @@ -32,12 +32,12 @@ public class NumberedOverwritingPartitionChunk implements OverwritingPartitio public NumberedOverwritingPartitionChunk(int chunkId, T object) { Preconditions.checkArgument( - chunkId >= ShardSpec.NON_ROOT_GEN_START_PARTITION_ID && chunkId < ShardSpec.NON_ROOT_GEN_END_PARTITION_ID, + chunkId >= PartitionIds.NON_ROOT_GEN_START_PARTITION_ID && chunkId < PartitionIds.NON_ROOT_GEN_END_PARTITION_ID, "partitionNum[%s] >= %s && partitionNum[%s] < %s", chunkId, - ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, chunkId, - ShardSpec.NON_ROOT_GEN_END_PARTITION_ID + PartitionIds.NON_ROOT_GEN_END_PARTITION_ID ); this.chunkId = chunkId; diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java index bbf4e5bc012a..f955e8f0f998 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java @@ -49,35 +49,38 @@ public NumberedOverwritingShardSpec( ) { Preconditions.checkArgument( - partitionId >= NON_ROOT_GEN_START_PARTITION_ID && partitionId < NON_ROOT_GEN_END_PARTITION_ID, + partitionId >= PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + && partitionId < PartitionIds.NON_ROOT_GEN_END_PARTITION_ID, "partitionNum[%s] >= %s && partitionNum[%s] < %s", partitionId, - NON_ROOT_GEN_START_PARTITION_ID, + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, partitionId, - NON_ROOT_GEN_END_PARTITION_ID + PartitionIds.NON_ROOT_GEN_END_PARTITION_ID ); Preconditions.checkArgument( - startRootPartitionId >= ROOT_GEN_START_PARTITION_ID && startRootPartitionId < ROOT_GEN_END_PARTITION_ID, + startRootPartitionId >= PartitionIds.ROOT_GEN_START_PARTITION_ID + && startRootPartitionId < PartitionIds.ROOT_GEN_END_PARTITION_ID, "startRootPartitionId[%s] >= %s && startRootPartitionId[%s] < %s", startRootPartitionId, - ROOT_GEN_START_PARTITION_ID, + PartitionIds.ROOT_GEN_START_PARTITION_ID, startRootPartitionId, - ROOT_GEN_END_PARTITION_ID + PartitionIds.ROOT_GEN_END_PARTITION_ID ); Preconditions.checkArgument( - endRootPartitionId >= ROOT_GEN_START_PARTITION_ID && endRootPartitionId < ROOT_GEN_END_PARTITION_ID, + endRootPartitionId >= PartitionIds.ROOT_GEN_START_PARTITION_ID + && endRootPartitionId < PartitionIds.ROOT_GEN_END_PARTITION_ID, "endRootPartitionId[%s] >= %s && endRootPartitionId[%s] < %s", endRootPartitionId, - ROOT_GEN_START_PARTITION_ID, + PartitionIds.ROOT_GEN_START_PARTITION_ID, endRootPartitionId, - ROOT_GEN_END_PARTITION_ID + PartitionIds.ROOT_GEN_END_PARTITION_ID ); Preconditions.checkArgument(minorVersion > 0, "minorVersion[%s] > 0", minorVersion); Preconditions.checkArgument( - atomicUpdateGroupSize > 0 || atomicUpdateGroupSize == ShardSpec.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE, + atomicUpdateGroupSize > 0 || atomicUpdateGroupSize == PartitionIds.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE, "atomicUpdateGroupSize[%s] > 0 or == %s", atomicUpdateGroupSize, - ShardSpec.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE + PartitionIds.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE ); this.partitionId = partitionId; @@ -99,7 +102,7 @@ public NumberedOverwritingShardSpec( startRootPartitionId, endRootPartitionId, minorVersion, - ShardSpec.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE + PartitionIds.UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE ); } 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 index 81ffb48b6853..07f4aa639242 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java @@ -66,7 +66,7 @@ public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPre { return new NumberedOverwritingShardSpec( specOfPreviousMaxPartitionId == null - ? ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + ? PartitionIds.NON_ROOT_GEN_START_PARTITION_ID : specOfPreviousMaxPartitionId.getPartitionNum() + 1, startRootPartitionId, endRootPartitionId, 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 index 634c8f5da4f2..8be997849bf9 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -21,9 +21,20 @@ 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.ISE; import org.apache.druid.timeline.Overshadowable; @@ -35,6 +46,7 @@ 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; import java.util.stream.Collectors; @@ -56,7 +68,7 @@ private enum State // start partitionId -> end partitionId -> minorVersion -> atomicUpdateGroup private final TreeMap>> standbyGroups; - private final TreeMap>> visibleGroup; // TODO: singleton navigable map + private final TreeMap>> visibleGroup; private final TreeMap>> overshadowedGroups; public OvershadowableManager() @@ -89,6 +101,19 @@ private TreeMap>> } } + private Short2ObjectSortedMap> 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 transitPartitionChunkState(AtomicUpdateGroup atomicUpdateGroup, State from, State to) { Preconditions.checkNotNull(atomicUpdateGroup, "atomicUpdateGroup"); @@ -204,7 +229,7 @@ private void handleAdd(AtomicUpdateGroup aug, State newStateOfAug) private void addTo(AtomicUpdateGroup aug, State state) { final AtomicUpdateGroup existing = getStateMap(state) - .computeIfAbsent(RootPartitionRange.of(aug), k -> new Short2ObjectRBTreeMap<>()) + .computeIfAbsent(RootPartitionRange.of(aug), k -> createMinorVersionToAugMap(state)) .put(aug.getMinorVersion(), aug); if (existing != null) { @@ -549,4 +574,181 @@ public String toString() '}'; } } + + private static class SingleEntryShort2ObjectSortedMap extends AbstractShort2ObjectSortedMap + { + private short key; + private V val; + + private SingleEntryShort2ObjectSortedMap() + { + key = -1; + val = null; + } + + @Override + public Short2ObjectSortedMap subMap(short fromKey, short toKey) + { + if (fromKey <= key && toKey > key) { + return this; + } else { + throw new IllegalArgumentException(); + } + } + + @Override + public Short2ObjectSortedMap headMap(short toKey) + { + if (toKey > key) { + return this; + } else { + throw new IllegalArgumentException(); + } + } + + @Override + public Short2ObjectSortedMap 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> 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 values() + { + return new AbstractObjectCollection() + { + @Override + public ObjectIterator 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) + { + final V existing = isEmpty() ? null : this.val; + this.key = key; + this.val = value; + return existing; + } + + @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 m) + { + if (!m.isEmpty()) { + if (m.size() == 1) { + final Map.Entry 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/OverwritingPartitionChunk.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java similarity index 69% rename from core/src/main/java/org/apache/druid/timeline/partition/OverwritingPartitionChunk.java rename to core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java index 86b2651e6ea4..2c81236b510f 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OverwritingPartitionChunk.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java @@ -19,7 +19,13 @@ package org.apache.druid.timeline.partition; -// TODO: rename -public interface OverwritingPartitionChunk extends PartitionChunk +public final class PartitionIds { + public static int ROOT_GEN_START_PARTITION_ID = 0; + public static int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive + public static int NON_ROOT_GEN_START_PARTITION_ID = 32768; + public static int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive + public static short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; + + private PartitionIds() {} } 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 5ddcc9e5154c..e8b527a7588c 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 @@ -42,13 +42,6 @@ }) public interface ShardSpec { - // TODO: move to somewhere..? - int ROOT_GEN_START_PARTITION_ID = 0; - int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive - int NON_ROOT_GEN_START_PARTITION_ID = 32768; - int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive - short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; - PartitionChunk createChunk(T obj); boolean isInChunk(long timestamp, InputRow inputRow); 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 2ad208114186..44d654825a26 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -34,7 +34,7 @@ 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.ShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.SingleElementPartitionChunk; import org.joda.time.DateTime; import org.joda.time.Days; @@ -2051,7 +2051,7 @@ private PartitionChunk makeNumberedOverwriting( int atomicUpdateGroupSize ) { - final int partitionNum = ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + partitionNumOrdinal; + final int partitionNum = PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + partitionNumOrdinal; return new NumberedOverwritingPartitionChunk<>( partitionNum, new OvershadowableInteger( 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 index ea06e63f3584..a2edfc6abeb5 100644 --- 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 @@ -32,8 +32,7 @@ import java.util.Objects; /** - * Lock for set of segments. Should be unique for (dataSource, interval, partitionId)?? - * TODO: is the above sentence true?? + * Lock for set of segments. Should be unique for (dataSource, interval, version, partitionId). */ public class SegmentLock implements TaskLock { 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 98a6499c2f8e..5bc33f67292c 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 @@ -53,12 +53,11 @@ * 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. *

- * This action implicitly acquires locks when it allocates segments. You do not have to acquire them beforehand, + * This action implicitly acquires segment locks when it allocates segments. You do not have to acquire them beforehand, * although you *do* have to release them yourself. *

- * If this action cannot acquire an appropriate lock, or if it cannot expand an existing segment set, it returns null. - * - * TODO: must be used with segment locking + * If this action cannot acquire an appropriate segment lock, or if it cannot expand an existing segment set, it returns + * null. */ public class SegmentAllocateAction implements TaskAction { @@ -275,7 +274,6 @@ private SegmentIdWithShardSpec tryAllocate( { // This action is always used by appending tasks, which cannot change the segment granularity of existing // dataSources. So, all lock requests should be segmentLock. - // TODO: don't use taskLockbox final LockResult lockResult = toolbox.getTaskLockbox().tryLock( task, new LockRequestForNewSegment( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java index f1e20dd19ca3..ed25ee41d08b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java @@ -45,7 +45,7 @@ public SegmentLockTryAcquireAction( @JsonProperty("lockType") TaskLockType type, @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, - @JsonProperty("partitionIds") Set partitionIds // TODO: IntSet + @JsonProperty("partitionIds") Set partitionIds ) { Preconditions.checkState(partitionIds != null && !partitionIds.isEmpty(), "partitionIds is empty"); 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 40a13dac9e85..f99b2dfc8a29 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 @@ -78,22 +78,22 @@ public abstract class AbstractTask implements Task private final Map context; @Nullable - private Map overwritingSegmentMetas; // TODO: rename? + private Map overwritingRootGenPartitions; @Nullable private Boolean changeSegmentGranularity; - public static class OverwritingSegmentMeta + public static class OverwritingRootGenerationPartitions { private final int startRootPartitionId; private final int endRootPartitionId; - private final short minorVersionForNewSegments; + private final short maxMinorVersion; - private OverwritingSegmentMeta(int startRootPartitionId, int endRootPartitionId, short minorVersionForNewSegments) + private OverwritingRootGenerationPartitions(int startRootPartitionId, int endRootPartitionId, short maxMinorVersion) { this.startRootPartitionId = startRootPartitionId; this.endRootPartitionId = endRootPartitionId; - this.minorVersionForNewSegments = minorVersionForNewSegments; + this.maxMinorVersion = maxMinorVersion; } public int getStartRootPartitionId() @@ -108,7 +108,7 @@ public int getEndRootPartitionId() public short getMinorVersionForNewSegments() { - return minorVersionForNewSegments; + return (short) (maxMinorVersion + 1); } } @@ -296,10 +296,10 @@ public Map getContext() return context; } - // TODO: remove this and check by findInputSegments returns empty? public abstract boolean requireLockInputSegments(); - public abstract List findInputSegments(TaskActionClient taskActionClient, List intervals) throws IOException; + public abstract List findInputSegments(TaskActionClient taskActionClient, List intervals) + throws IOException; public abstract boolean changeSegmentGranularity(List intervalOfExistingSegments); @@ -323,22 +323,22 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i throws IOException { if (requireLockInputSegments()) { - final List intervalsToFindInput = new ArrayList<>(intervals); - if (overwritingSegmentMetas != null) { - intervalsToFindInput.removeAll(overwritingSegmentMetas.keySet()); - } - // TODO: check changeSegmentGranularity and get timeChunkLock here - // TODO: race - a new segment can be added after findInputSegments. change to lockAllSegmentsInIntervals - if (!intervalsToFindInput.isEmpty()) { - return tryLockWithSegments(client, findInputSegments(client, intervalsToFindInput)); + // This method finds segments falling in all given intervals and then tries to lock those segments. + // Thus, there might be a race between calling findInputSegments() and tryLockWithSegments(), + // 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 findInputSegments() call, it may missing those + // segments. This is usually fine, but if you want to avoid this, you should use timeChunk lock instead. + if (!intervals.isEmpty()) { + return tryLockWithSegments(client, findInputSegments(client, intervals)); } else { return true; } } else { changeSegmentGranularity = false; - overwritingSegmentMetas = Collections.emptyMap(); + overwritingRootGenPartitions = Collections.emptyMap(); return true; } } @@ -349,7 +349,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) if (requireLockInputSegments()) { if (segments.isEmpty()) { changeSegmentGranularity = false; - overwritingSegmentMetas = Collections.emptyMap(); + overwritingRootGenPartitions = Collections.emptyMap(); return true; } @@ -358,7 +358,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) changeSegmentGranularity = changeSegmentGranularity(intervals); if (changeSegmentGranularity) { - overwritingSegmentMetas = Collections.emptyMap(); + overwritingRootGenPartitions = Collections.emptyMap(); // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined final Set uniqueIntervals = new HashSet<>(); for (Interval interval : JodaUtils.condenseIntervals(intervals)) { @@ -403,7 +403,12 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) .map(s -> s.getShardSpec().getPartitionNum()) .collect(Collectors.toSet()); final List lockResults = client.submit( - new SegmentLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval, entry.getValue().get(0).getVersion(), partitionIds) + new SegmentLockTryAcquireAction( + TaskLockType.EXCLUSIVE, + interval, + entry.getValue().get(0).getVersion(), + partitionIds + ) ); if (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk())) { // TODO: unlock @@ -414,7 +419,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) } } else { changeSegmentGranularity = false; - overwritingSegmentMetas = Collections.emptyMap(); + overwritingRootGenPartitions = Collections.emptyMap(); return true; } } @@ -476,15 +481,15 @@ private void verifyAndFindRootPartitionRangeAndMinorVersion(List in .max() .orElseThrow(() -> new ISE("Empty inputSegments")); - if (overwritingSegmentMetas == null) { - overwritingSegmentMetas = new HashMap<>(); + if (overwritingRootGenPartitions == null) { + overwritingRootGenPartitions = new HashMap<>(); } - overwritingSegmentMetas.put( + overwritingRootGenPartitions.put( interval, - new OverwritingSegmentMeta( + new OverwritingRootGenerationPartitions( inputSegments.get(0).getStartRootPartitionId(), inputSegments.get(inputSegments.size() - 1).getEndRootPartitionId(), - (short) (prevMaxMinorVersion + 1) + prevMaxMinorVersion ) ); } @@ -494,22 +499,22 @@ protected boolean isChangeSegmentGranularity() return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); } - Map getAllOverwritingSegmentMeta() + Map getAllOverwritingSegmentMeta() { - Preconditions.checkNotNull(overwritingSegmentMetas, "overwritingSegmentMetas is not initialized"); - return overwritingSegmentMetas; + Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); + return Collections.unmodifiableMap(overwritingRootGenPartitions); } @Nullable - public OverwritingSegmentMeta getOverwritingSegmentMeta(Interval interval) + public OverwritingRootGenerationPartitions getOverwritingSegmentMeta(Interval interval) { - Preconditions.checkNotNull(overwritingSegmentMetas, "overwritingSegmentMetas is not initialized"); - return overwritingSegmentMetas.get(interval); + Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); + return overwritingRootGenPartitions.get(interval); } public boolean isOverwriteMode() { - Preconditions.checkNotNull(overwritingSegmentMetas, "overwritingSegmentMetas is not initialized"); - return !overwritingSegmentMetas.isEmpty(); + Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); + return !overwritingRootGenPartitions.isEmpty(); } } 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 3cf0f6706b4d..d3bdc82107dd 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 @@ -717,7 +717,7 @@ private Pair createShardSpecFactoryForBestEffortRollu ) { if (overwrite && !isChangeSegmentGranularity()) { - final OverwritingSegmentMeta overwritingSegmentMeta = Preconditions.checkNotNull( + final OverwritingRootGenerationPartitions overwritingSegmentMeta = Preconditions.checkNotNull( getOverwritingSegmentMeta(interval), "Can't find overwritingSegmentMeta for interval[%s]", interval @@ -843,7 +843,14 @@ private IndexTaskSegmentAllocator createSegmentAllocator( if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { return new CachingRemoteSegmentAllocator(toolbox, getId(), allocateSpec); } else { - return new RemoteSegmentAllocator(toolbox, getId(), dataSchema, isOverwriteMode(), isChangeSegmentGranularity(), getAllOverwritingSegmentMeta()); + return new RemoteSegmentAllocator( + toolbox, + getId(), + dataSchema, + isOverwriteMode(), + isChangeSegmentGranularity(), + getAllOverwritingSegmentMeta() + ); } } else { // We use the timeChunk lock and don't have to ask the overlord to create segmentIds. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java index 0ec1ada6cd43..b5fad4d92b5c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java @@ -23,7 +23,7 @@ import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.task.AbstractTask.OverwritingSegmentMeta; +import org.apache.druid.indexing.common.task.AbstractTask.OverwritingRootGenerationPartitions; import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -47,7 +47,7 @@ public RemoteSegmentAllocator( DataSchema dataSchema, boolean isOverwriteMode, boolean isChangeSegmentGranularity, - Map overwritingSegmentMetaMap + Map overwritingSegmentMetaMap ) { this.taskId = taskId; @@ -61,7 +61,7 @@ public RemoteSegmentAllocator( .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); final ShardSpecFactory shardSpecFactory; if (isOverwriteMode && !isChangeSegmentGranularity) { - final OverwritingSegmentMeta overwritingSegmentMeta = overwritingSegmentMetaMap.get(interval); + final OverwritingRootGenerationPartitions overwritingSegmentMeta = overwritingSegmentMetaMap.get(interval); if (overwritingSegmentMeta == null) { throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 7b9189995639..f435ed4817d1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -296,7 +296,7 @@ private SegmentAllocator createSegmentAllocator() .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); final ShardSpecFactory shardSpecFactory; if (isOverwriteMode() && !isChangeSegmentGranularity()) { - final OverwritingSegmentMeta overwritingSegmentMeta = getOverwritingSegmentMeta(interval); + final OverwritingRootGenerationPartitions overwritingSegmentMeta = getOverwritingSegmentMeta(interval); if (overwritingSegmentMeta == null) { throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 9d2e4e20caf5..8fb3eefeec75 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -56,7 +56,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedOverwritingShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -250,7 +250,10 @@ public void testRunCompactionTwiceWithKeepSegmentGranularity() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); + Assert.assertEquals( + new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), + segments.get(i).getShardSpec() + ); } final CompactionTask compactionTask2 = builder @@ -267,7 +270,10 @@ public void testRunCompactionTwiceWithKeepSegmentGranularity() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + 1, 0, 2, (short) 2, (short) 1), segments.get(i).getShardSpec()); + Assert.assertEquals( + new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, 0, 2, (short) 2, (short) 1), + segments.get(i).getShardSpec() + ); } } @@ -352,7 +358,10 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); + Assert.assertEquals( + new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), + segments.get(i).getShardSpec() + ); } } @@ -487,8 +496,14 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception Assert.assertEquals(6, segments.size()); for (int i = 0; i < 6; i++) { - Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), segments.get(i).getShardSpec()); + Assert.assertEquals( + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), + segments.get(i).getInterval() + ); + Assert.assertEquals( + new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), + segments.get(i).getShardSpec() + ); } final Pair> compactionResult = compactionFuture.get(); @@ -542,8 +557,14 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio Assert.assertEquals(6, segments.size()); for (int i = 0; i < 6; i++) { - Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedOverwritingShardSpec(ShardSpec.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), segments.get(i).getShardSpec()); + Assert.assertEquals( + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), + segments.get(i).getInterval() + ); + Assert.assertEquals( + new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), + segments.get(i).getShardSpec() + ); } final Pair> compactionResult = compactionFuture.get(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 975aab01603a..d810b8bacf30 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -60,7 +60,7 @@ import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; -import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.ShardSpecFactory; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -962,7 +962,7 @@ public void testRequestForNewSegmentWithSegmentLock() final SegmentLock segmentLock = (SegmentLock) lock; Assert.assertEquals(expectedPartitionId++, segmentLock.getPartitionId()); if (expectedPartitionId == 3) { - expectedPartitionId = ShardSpec.NON_ROOT_GEN_START_PARTITION_ID; + expectedPartitionId = PartitionIds.NON_ROOT_GEN_START_PARTITION_ID; } } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index e7da678c4518..c2bc840dac94 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -42,7 +42,7 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.timeline.partition.PartitionChunk; -import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -979,7 +979,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce Assert.assertEquals( StringUtils.format( "ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version%s", - "_" + (i + ShardSpec.NON_ROOT_GEN_START_PARTITION_ID) + "_" + (i + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID) ), identifier.toString() ); @@ -1022,7 +1022,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce Collections.emptyList(), Collections.emptyList(), new NumberedOverwritingShardSpec( - 9 + ShardSpec.NON_ROOT_GEN_START_PARTITION_ID, + 9 + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 1, (short) 9, From a2970850ba56479f25311b18ca9f3c55cc5e5f16 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 24 Apr 2019 17:48:44 -0700 Subject: [PATCH 10/52] resolving todos --- .../NewestSegmentFirstPolicyBenchmark.java | 2 +- .../segment/loading/DataSegmentPusher.java | 2 +- .../apache/druid/timeline/DataSegment.java | 9 +- .../apache/druid/timeline/Overshadowable.java | 2 +- .../timeline/VersionedIntervalTimeline.java | 2 +- .../timeline/partition/AtomicUpdateGroup.java | 7 + .../druid/timeline/DataSegmentTest.java | 2 +- .../VersionedIntervalTimelineTest.java | 587 +++++++++--------- .../partition/IntegerPartitionChunkTest.java | 6 + .../storage/azure/AzureDataSegmentPusher.java | 2 +- .../MaterializedViewSupervisor.java | 2 +- .../storage/hdfs/HdfsDataSegmentPusher.java | 2 +- .../druid/indexer/IndexGeneratorJobTest.java | 2 +- .../actions/SegmentLockReleaseAction.java | 81 +++ .../actions/SegmentMetadataUpdateAction.java | 14 +- .../common/actions/SegmentNukeAction.java | 14 +- .../SegmentTransactionalInsertAction.java | 12 +- .../actions/TaskActionPreconditions.java | 4 +- .../indexing/common/task/AbstractTask.java | 33 +- .../indexing/common/task/ArchiveTask.java | 4 +- .../common/task/CachingSegmentAllocator.java | 4 +- .../indexing/common/task/CompactionTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 1 - .../task/IndexTaskSegmentAllocator.java | 6 +- .../druid/indexing/common/task/MoveTask.java | 4 +- .../common/task/RealtimeIndexTask.java | 5 +- .../indexing/common/task/RestoreTask.java | 4 +- .../batch/parallel/ParallelIndexSubTask.java | 1 - .../IngestSegmentFirehoseFactory.java | 4 +- .../druid/indexing/overlord/TaskLockbox.java | 55 +- .../actions/SegmentInsertActionTest.java | 4 +- ...penderatorDriverRealtimeIndexTaskTest.java | 18 +- .../common/task/RealtimeIndexTaskTest.java | 10 +- .../overlord/TaskLockBoxConcurrencyTest.java | 9 +- .../indexing/overlord/TaskLockboxTest.java | 10 +- .../segment/ReferenceCountingSegment.java | 6 + .../druid/query/QueryRunnerTestHelper.java | 1 - .../druid/segment/SchemalessIndexTest.java | 12 +- .../apache/druid/client/BrokerServerView.java | 6 +- .../druid/client/CoordinatorServerView.java | 6 +- .../apache/druid/client/SegmentLoadInfo.java | 6 + .../druid/client/selector/ServerSelector.java | 6 + .../SQLMetadataStorageUpdaterJobHandler.java | 2 +- .../IndexerSQLMetadataStorageCoordinator.java | 2 +- .../metadata/SQLMetadataSegmentPublisher.java | 2 +- .../appenderator/AppenderatorImpl.java | 6 +- .../appenderator/SegmentIdWithShardSpec.java | 2 +- ...oordinatorBasedSegmentHandoffNotifier.java | 2 +- .../realtime/plumber/RealtimePlumber.java | 6 +- .../druid/segment/realtime/plumber/Sink.java | 11 +- .../apache/druid/server/SegmentManager.java | 8 +- .../DruidCoordinatorRuntimeParams.java | 2 +- .../DruidCoordinatorCleanupOvershadowed.java | 2 +- .../helper/DruidCoordinatorRuleRunner.java | 2 +- .../server/http/DataSourcesResource.java | 2 +- .../client/CachingClusteredClientTest.java | 6 +- .../appenderator/TestUsedSegmentChecker.java | 2 +- .../druid/server/SegmentManagerTest.java | 2 +- .../DruidCoordinatorSegmentCompactorTest.java | 20 +- .../server/shard/NumberedShardSpecTest.java | 6 + .../sql/calcite/schema/SystemSchema.java | 4 +- .../SpecificSegmentsQuerySegmentWalker.java | 2 +- 62 files changed, 580 insertions(+), 480 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java diff --git a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index d84bbfd69249..7e022a7f35e9 100644 --- a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -128,7 +128,7 @@ public void setup() 0, segmentSizeBytes ); - timeline.add(segment.getInterval(), segment.getVersion(), shardSpec.createChunk(segment)); + timeline.add(segment.getInterval(), segment.getMajorVersion(), shardSpec.createChunk(segment)); } } diff --git a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java index a1da4f89ec39..cdd474a579cf 100644 --- a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java +++ b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java @@ -108,7 +108,7 @@ static String getDefaultStorageDir(DataSegment segment, boolean useUniquePath) return JOINER.join( segment.getDataSource(), StringUtils.format("%s_%s", segment.getInterval().getStart(), segment.getInterval().getEnd()), - segment.getVersion(), + segment.getMajorVersion(), segment.getShardSpec().getPartitionNum(), useUniquePath ? generateUniquePath() : null ); 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 fb4a876d18f4..bc00f098183d 100644 --- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -229,8 +229,9 @@ public Map getLoadSpec() return loadSpec; } - @JsonProperty - public String getVersion() + @JsonProperty("version") + @Override + public String getMajorVersion() { return id.getVersion(); } @@ -320,7 +321,7 @@ private boolean includeRootPartitions(DataSegment other) public SegmentDescriptor toDescriptor() { - return new SegmentDescriptor(getInterval(), getVersion(), shardSpec.getPartitionNum()); + return new SegmentDescriptor(getInterval(), getMajorVersion(), shardSpec.getPartitionNum()); } public DataSegment withLoadSpec(Map loadSpec) @@ -451,7 +452,7 @@ public Builder(DataSegment segment) { this.dataSource = segment.getDataSource(); this.interval = segment.getInterval(); - this.version = segment.getVersion(); + this.version = segment.getMajorVersion(); this.loadSpec = segment.getLoadSpec(); this.dimensions = segment.getDimensions(); this.metrics = segment.getMetrics(); diff --git a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java index 1a9790e12157..2cdac31fcca6 100644 --- a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java +++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java @@ -67,7 +67,7 @@ default short getEndRootPartitionIdAsShort() return (short) getEndRootPartitionId(); } - // TODO: majorVersion? + String getMajorVersion(); short getMinorVersion(); 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 551b8bbc9b28..e3f967b458b3 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -114,7 +114,7 @@ public static void addSegments( timeline.addAll( Iterators.transform(segments, segment -> segment.getShardSpec().createChunk(segment)), DataSegment::getInterval, - DataSegment::getVersion + DataSegment::getMajorVersion ); } 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 index fe76daba1f97..2f1f3e28cb35 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -132,6 +132,13 @@ public short getEndRootPartitionIdAsShort() return chunks.get(0).getObject().getEndRootPartitionIdAsShort(); } + @Override + public String getMajorVersion() + { + Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); + return chunks.get(0).getObject().getMajorVersion(); + } + @Override public short getMinorVersion() { 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 bc3cf902b9ed..ef3d692c4904 100644 --- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -149,7 +149,7 @@ public void testV1Serialization() throws Exception Assert.assertEquals(segment.getDataSource(), deserializedSegment.getDataSource()); Assert.assertEquals(segment.getInterval(), deserializedSegment.getInterval()); - Assert.assertEquals(segment.getVersion(), deserializedSegment.getVersion()); + Assert.assertEquals(segment.getMajorVersion(), deserializedSegment.getMajorVersion()); Assert.assertEquals(segment.getLoadSpec(), deserializedSegment.getLoadSpec()); Assert.assertEquals(segment.getDimensions(), deserializedSegment.getDimensions()); Assert.assertEquals(segment.getMetrics(), deserializedSegment.getMetrics()); 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 44d654825a26..2d89ce25b7dc 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -74,8 +74,8 @@ public void setUp() 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, new OvershadowableInteger(0, 20))); - add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger(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); @@ -98,8 +98,8 @@ public void testApril() 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( @@ -116,12 +116,12 @@ public void testApril2() 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( @@ -137,8 +137,8 @@ public void testApril3() 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( @@ -172,7 +172,7 @@ public void testMay() @Test public void testMay2() { - Assert.assertNotNull(timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle(9))); + 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), @@ -187,12 +187,12 @@ public void testMay2() 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( @@ -244,22 +244,22 @@ public void testRemove() 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") ); @@ -283,7 +283,7 @@ public void testFindEntryWithOverlap() 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") ); } @@ -297,8 +297,8 @@ public void testPartitioning() createExpected( "2011-10-02/2011-10-03", "3", Arrays.asList( - IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger(0, 20)), - IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger(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), @@ -314,30 +314,30 @@ public void testPartialPartitionNotReturned() { testRemove(); - add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger(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.findFullyOvershadowed().isEmpty()); - add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger(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.findFullyOvershadowed().isEmpty()); - add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger(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, new OvershadowableInteger(0, 60)), - IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger(1, 61)), - IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger(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)) ) ) ), @@ -351,13 +351,13 @@ public void testIncompletePartitionDoesNotOvershadow() { testRemove(); - add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger(0, 60))); + 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, new OvershadowableInteger(1, 61))); + 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, new OvershadowableInteger(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) @@ -371,7 +371,12 @@ public void testRemovePartitionMakesIncomplete() { testIncompletePartitionDoesNotOvershadow(); - final IntegerPartitionChunk chunk = IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger(0, 60)); + final IntegerPartitionChunk 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)), @@ -392,8 +397,8 @@ public void testInsertAndRemoveSameThingsion() ); 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( @@ -411,8 +416,8 @@ public void testInsertAndRemoveSameThingsion() ); 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( @@ -1172,7 +1177,7 @@ public void testOverlapAndRemove() 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( @@ -1191,7 +1196,7 @@ public void testOverlapAndRemove2() 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( @@ -1212,8 +1217,8 @@ public void testOverlapAndRemove3() 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( @@ -1233,7 +1238,7 @@ public void testOverlapAndRemove4() 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( @@ -1252,7 +1257,7 @@ public void testOverlapAndRemove5() 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( @@ -1268,11 +1273,11 @@ public void testRemoveSomethingDontHave() { 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)) ); } @@ -1286,7 +1291,7 @@ public void testRemoveNothingBacking() 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( @@ -1510,30 +1515,30 @@ public void testRemoveIncompleteKeepsComplete() { timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger(0, 77))); - add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(1, 88))); - add("2011-04-01/2011-04-02", "2", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger(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, new OvershadowableInteger(0, 77)), - IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(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, new OvershadowableInteger(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, new OvershadowableInteger(0, 77)), - IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(1, 88)) + IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("1", 0, 77)), + IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger("1", 1, 88)) ) ) ), @@ -1543,7 +1548,7 @@ public void testRemoveIncompleteKeepsComplete() Sets.newHashSet( createExpected("2011-04-01/2011-04-02", "2", Collections.singletonList( - IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger(0, 99)) + IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("2", 0, 99)) ) ) ), @@ -1556,8 +1561,8 @@ public void testRemoveIncompleteKeepsComplete() ImmutableList.of( createExpected("2011-04-01/2011-04-02", "1", Arrays.asList( - IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger(0, 77)), - IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger(1, 88)) + IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("1", 0, 77)), + IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger("1", 1, 88)) ) ) ), @@ -1570,64 +1575,64 @@ public void testIsOvershadowedWithNonOverlappingSegmentsInTimeline() { timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-07", "1", makeSingle(1)); - add("2011-04-07/2011-04-09", "1", makeSingle(1)); - - add("2011-04-15/2011-04-17", "1", makeSingle(1)); - add("2011-04-17/2011-04-19", "1", makeSingle(1)); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "1", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "1", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "1", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "1", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "2", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "2", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "2", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "2", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-30"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-08"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-07/2011-04-30"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-08/2011-04-30"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-15"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-17"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-19"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-09/2011-04-30"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-16"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-17"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-18"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-19"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-20"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-30"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-19/2011-04-20"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0", new OvershadowableInteger(0, 1))); + 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 @@ -1635,83 +1640,83 @@ public void testIsOvershadowedWithOverlappingSegmentsInTimeline() { timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-09", "11", makeSingle(1)); - add("2011-04-07/2011-04-11", "12", makeSingle(1)); - - add("2011-04-15/2011-04-19", "12", makeSingle(1)); - add("2011-04-17/2011-04-21", "11", makeSingle(1)); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-06"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-07"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-08"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-11"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-30"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "12", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "12", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "12", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "12", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "12", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "12", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-06"), "13", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-07"), "13", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-08"), "13", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-09"), "13", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-10"), "13", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-11"), "13", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-12"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-15"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-16"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-17"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-18"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-19"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-20"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-21"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-05/2011-04-22"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-07"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-08"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-09"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-10"), "0", new OvershadowableInteger(0, 1))); - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-11"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-12"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-15"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-16"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-17"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-18"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-19"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-20"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-21"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-06/2011-04-22"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-15"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-16"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-17"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-18"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-19"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-20"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-21"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-12/2011-04-22"), "0", new OvershadowableInteger(0, 1))); - - Assert.assertTrue(timeline.isOvershadowed(Intervals.of("2011-04-15/2011-04-21"), "0", new OvershadowableInteger(0, 1))); - Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-21/2011-04-22"), "0", new OvershadowableInteger(0, 1))); + 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 @@ -1719,19 +1724,19 @@ public void testOvershadowedByReference() { timeline = makeStringIntegerTimeline(); - add("2019-01-01/2019-01-02", "0", makeNumbered(0, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + 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, 1, 0, 3, 1, 2)); - add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(1, 1, 0, 3, 1, 2)); + 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), makeNumbered(1, 0), makeNumbered(2, 0)) + ImmutableList.of(makeNumbered("0", 0, 0), makeNumbered("0", 1, 0), makeNumbered("0", 2, 0)) ) ), timeline.findFullyOvershadowed() @@ -1744,21 +1749,21 @@ public void testOvershadowedByReferenceChain() timeline = makeStringIntegerTimeline(); // 2019-01-01/2019-01-02 - add("2019-01-01/2019-01-02", "0", makeNumbered(0, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + 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)); - add("2019-01-02/2019-01-03", "0", makeNumbered(1, 0)); + 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, 1, 0, 3, 1, 2)); - add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(1, 1, 0, 3, 1, 2)); + 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(2, 2, 0, 3, 2, 2)); - add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(3, 2, 0, 3, 2, 2)); + 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( @@ -1766,11 +1771,11 @@ public void testOvershadowedByReferenceChain() "2019-01-01/2019-01-02", "0", ImmutableList.of( - makeNumbered(0, 0), - makeNumbered(1, 0), - makeNumbered(2, 0), - makeNumberedOverwriting(0, 1, 0, 3, 1, 2), - makeNumberedOverwriting(1, 1, 0, 3, 1, 2) + 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) ) ) ), @@ -1784,21 +1789,21 @@ public void testOvershadowedByReferenceAndThenVersion() timeline = makeStringIntegerTimeline(); // 2019-01-01/2019-01-02 - add("2019-01-01/2019-01-02", "0", makeNumbered(0, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + 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)); - add("2019-01-02/2019-01-03", "0", makeNumbered(1, 0)); + 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, 1, 0, 3, 1, 2)); - add("2019-01-01/2019-01-02", "0", makeNumberedOverwriting(1, 1, 0, 3, 1, 2)); + 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(0, 0)); - add("2019-01-01/2019-01-02", "1", makeNumbered(1, 0)); + 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( @@ -1806,11 +1811,11 @@ public void testOvershadowedByReferenceAndThenVersion() "2019-01-01/2019-01-02", "0", ImmutableList.of( - makeNumbered(0, 0), - makeNumbered(1, 0), - makeNumbered(2, 0), - makeNumberedOverwriting(0, 1, 0, 3, 1, 2), - makeNumberedOverwriting(1, 1, 0, 3, 1, 2) + 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) ) ) ), @@ -1824,22 +1829,22 @@ public void testOvershadowedByVersionAndThenReference() timeline = makeStringIntegerTimeline(); // 2019-01-01/2019-01-02 - add("2019-01-01/2019-01-02", "0", makeNumbered(0, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(1, 0)); - add("2019-01-01/2019-01-02", "0", makeNumbered(2, 0)); + 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)); - add("2019-01-02/2019-01-03", "0", makeNumbered(1, 0)); + 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(0, 0)); - add("2019-01-01/2019-01-02", "1", makeNumbered(1, 0)); + 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(0, 1, 0, 2, 1, 3)); - add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(1, 1, 0, 2, 1, 3)); - add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(2, 1, 0, 2, 1, 3)); + 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( @@ -1847,17 +1852,17 @@ public void testOvershadowedByVersionAndThenReference() "2019-01-01/2019-01-02", "0", ImmutableList.of( - makeNumbered(0, 0), - makeNumbered(1, 0), - makeNumbered(2, 0) + makeNumbered("0", 0, 0), + makeNumbered("0", 1, 0), + makeNumbered("0", 2, 0) ) ), makeTimelineObjectHolder( "2019-01-01/2019-01-02", "1", ImmutableList.of( - makeNumbered(0, 0), - makeNumbered(1, 0) + makeNumbered("1", 0, 0), + makeNumbered("1", 1, 0) ) ) ), @@ -1872,23 +1877,23 @@ public void testFallbackOnMissingSegment() final Interval interval = Intervals.of("2019-01-01/2019-01-02"); - add(interval, "0", makeNumbered(0, 0)); - add(interval, "0", makeNumbered(1, 0)); - add(interval, "0", makeNumbered(2, 0)); + 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(0, 0)); - add(interval, "1", makeNumbered(1, 0)); + 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(0, 1, 0, 2, 1, 3)); - add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(1, 1, 0, 2, 1, 3)); - add("2019-01-01/2019-01-02", "1", makeNumberedOverwriting(2, 1, 0, 2, 1, 3)); + 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(2, 1, 0, 2, 1, 3) + makeNumberedOverwriting("1", 2, 1, 0, 2, 1, 3) ); final List> holders = timeline.lookup(interval); @@ -1900,10 +1905,10 @@ public void testFallbackOnMissingSegment() "1", new PartitionHolder<>( ImmutableList.of( - makeNumbered(0, 0), - makeNumbered(1, 0), - makeNumberedOverwriting(0, 1, 0, 2, 1, 3), - makeNumberedOverwriting(1, 1, 0, 2, 1, 3) + makeNumbered("1", 0, 0), + makeNumbered("1", 1, 0), + makeNumberedOverwriting("1", 0, 1, 0, 2, 1, 3), + makeNumberedOverwriting("1", 1, 1, 0, 2, 1, 3) ) ) ) @@ -1917,9 +1922,9 @@ public void testAddSameChunkToFullAtomicUpdateGroup() { timeline = makeStringIntegerTimeline(); final Interval interval = Intervals.of("2019-01-01/2019-01-02"); - add(interval, "0", makeNumbered(0, 0)); - add(interval, "0", makeNumberedOverwriting(0, 0, 0, 1, 1, 1)); - add(interval, "0", makeNumbered(0, 1)); + 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> overshadowed = timeline.findFullyOvershadowed(); Assert.assertEquals( @@ -1927,7 +1932,7 @@ public void testAddSameChunkToFullAtomicUpdateGroup() new TimelineObjectHolder<>( interval, "0", - new PartitionHolder<>(ImmutableList.of(makeNumbered(0, 1))) + new PartitionHolder<>(ImmutableList.of(makeNumbered("0", 0, 1))) ) ), overshadowed @@ -1939,10 +1944,10 @@ public void testOvershadowMultipleStandbyAtomicUpdateGroup() { timeline = makeStringIntegerTimeline(); final Interval interval = Intervals.of("2019-01-01/2019-01-02"); - add(interval, "0", makeNumberedOverwriting(0, 0, 0, 1, 1, 2)); - add(interval, "0", makeNumberedOverwriting(1, 0, 0, 1, 2, 2)); - add(interval, "0", makeNumberedOverwriting(2, 0, 0, 1, 3, 2)); // <-- full atomicUpdateGroup - add(interval, "0", makeNumberedOverwriting(3, 1, 0, 1, 3, 2)); // <-- full atomicUpdateGroup + 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> overshadowed = timeline.findFullyOvershadowed(); Assert.assertEquals( @@ -1952,8 +1957,8 @@ public void testOvershadowMultipleStandbyAtomicUpdateGroup() "0", new PartitionHolder<>( ImmutableList.of( - makeNumberedOverwriting(0, 0, 0, 1, 1, 2), - makeNumberedOverwriting(1, 0, 0, 1, 2, 2) + makeNumberedOverwriting("0", 0, 0, 0, 1, 1, 2), + makeNumberedOverwriting("0", 1, 0, 0, 1, 2, 2) ) ) ) @@ -1967,21 +1972,21 @@ public void testIsOvershadowedForOverwritingSegments() { timeline = makeStringIntegerTimeline(); final Interval interval = Intervals.of("2019-01-01/2019-01-02"); - add(interval, "0", makeNumberedOverwriting(0, 0, 5, 10, 10, 1)); + 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(i + 5, 0).getObject())); + Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumbered("0", i + 5, 0).getObject())); } - Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumbered(4, 0).getObject())); - Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumbered(11, 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(1, 0, 5, 6, 5, 2).getObject())); - Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 7, 8, 5, 2).getObject())); - Assert.assertTrue(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 8, 10, 5, 2).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(1, 0, 5, 10, 12, 2).getObject())); - Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting(1, 0, 4, 15, 12, 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 makeTimelineObjectHolder( @@ -2007,7 +2012,7 @@ private Pair>> cre return createExpected( intervalString, version, - Collections.singletonList(makeSingle(value)) + Collections.singletonList(makeSingle(version, value)) ); } @@ -2023,26 +2028,27 @@ private Pair>> cre ); } - private PartitionChunk makeSingle(int value) + private PartitionChunk makeSingle(String majorVersion, int value) { - return makeSingle(0, value); + return makeSingle(majorVersion, 0, value); } - private PartitionChunk makeSingle(int partitionNum, int val) + private PartitionChunk makeSingle(String majorVersion, int partitionNum, int val) { - return new SingleElementPartitionChunk<>(new OvershadowableInteger(partitionNum, val)); + return new SingleElementPartitionChunk<>(new OvershadowableInteger(majorVersion, partitionNum, val)); } - private PartitionChunk makeNumbered(int partitionNum, int val) + private PartitionChunk makeNumbered(String majorVersion, int partitionNum, int val) { return new NumberedPartitionChunk<>( partitionNum, 0, - new OvershadowableInteger(partitionNum, val) + new OvershadowableInteger(majorVersion, partitionNum, val) ); } private PartitionChunk makeNumberedOverwriting( + String majorVersion, int partitionNumOrdinal, int val, int startRootPartitionId, @@ -2055,6 +2061,7 @@ private PartitionChunk makeNumberedOverwriting( return new NumberedOverwritingPartitionChunk<>( partitionNum, new OvershadowableInteger( + majorVersion, partitionNum, val, startRootPartitionId, @@ -2072,7 +2079,7 @@ private void add(String interval, String version, Integer value) 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 value) @@ -2141,6 +2148,7 @@ private VersionedIntervalTimeline makeStringInteg private static class OvershadowableInteger implements Overshadowable { + private final String majorVersion; private final int partitionNum; private final int val; private final int startRootPartitionId; @@ -2148,12 +2156,13 @@ private static class OvershadowableInteger implements Overshadowable, Map>> getVersion Map> segments = new HashMap<>(); for (DataSegment segment : snapshot) { Interval interval = segment.getInterval(); - versions.put(interval, segment.getVersion()); + versions.put(interval, segment.getMajorVersion()); segments.putIfAbsent(interval, new ArrayList<>()); segments.get(interval).add(segment); } diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java index fd479768530c..76d39c3fa1ed 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java @@ -203,7 +203,7 @@ public String getStorageDir(DataSegment segment, boolean useUniquePath) segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()), segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime()) ), - segment.getVersion().replace(':', '_') + segment.getMajorVersion().replace(':', '_') ); } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index 743c03cf57bc..e28337eac8cd 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -650,7 +650,7 @@ private void verifyJob(IndexGeneratorJob job) throws IOException final DataSegment dataSegment = segments.get(i); final File indexZip = indexFiles.get(i); - Assert.assertEquals(config.getSchema().getTuningConfig().getVersion(), dataSegment.getVersion()); + Assert.assertEquals(config.getSchema().getTuningConfig().getVersion(), dataSegment.getMajorVersion()); Assert.assertEquals("local", dataSegment.getLoadSpec().get("type")); Assert.assertEquals(indexZip.getCanonicalPath(), dataSegment.getLoadSpec().get("path")); Assert.assertEquals(Integer.valueOf(9), dataSegment.getBinaryVersion()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java new file mode 100644 index 000000000000..b4d4e95db4c2 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.indexing.common.task.Task; +import org.joda.time.Interval; + +public class SegmentLockReleaseAction implements TaskAction +{ + private final Interval interval; + private final int partitionId; + + @JsonCreator + public SegmentLockReleaseAction(@JsonProperty Interval interval, @JsonProperty int partitionId) + { + this.interval = interval; + this.partitionId = partitionId; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public int getPartitionId() + { + return partitionId; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public Void perform(Task task, TaskActionToolbox toolbox) + { + toolbox.getTaskLockbox().unlock(task, interval, partitionId); + return null; + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return "SegmentLockReleaseAction{" + + "interval=" + interval + + ", partitionId=" + partitionId + + '}'; + } +} 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 0a8500261f43..fcfa74843a16 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,13 +32,9 @@ 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.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; public class SegmentMetadataUpdateAction implements TaskAction { @@ -70,16 +66,10 @@ public Void perform(Task task, TaskActionToolbox toolbox) { TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - final Map> intervalToPartitionIds = new HashMap<>(); - for (DataSegment segment : segments) { - intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) - .add(segment.getShardSpec().getPartitionNum()); - } - try { toolbox.getTaskLockbox().doInCriticalSection( task, - intervalToPartitionIds, + 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 d3dd0ddd8bd9..7b7d48605f30 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,13 +32,9 @@ 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.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; public class SegmentNukeAction implements TaskAction { @@ -72,16 +68,10 @@ public Void perform(Task task, TaskActionToolbox toolbox) { TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - final Map> intervalToPartitionIds = new HashMap<>(); - for (DataSegment segment : segments) { - intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) - .add(segment.getShardSpec().getPartitionNum()); - } - try { toolbox.getTaskLockbox().doInCriticalSection( task, - intervalToPartitionIds, + 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 ebc4acb14bed..ba44154b3948 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 @@ -37,6 +37,7 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -106,12 +107,11 @@ public TypeReference getReturnTypeReference() @Override public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) { - // TODO: move this to lock checking in critical section TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); - final Map> intervalToPartitionIds = new HashMap<>(); + final Map> intervalToPartitionIds = new HashMap<>(); for (DataSegment segment : segments) { - intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) + intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new HashSet<>()) .add(segment.getShardSpec().getPartitionNum()); } @@ -119,7 +119,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) try { retVal = toolbox.getTaskLockbox().doInCriticalSection( task, - intervalToPartitionIds, + new ArrayList<>(intervalToPartitionIds.keySet()), CriticalAction.builder() .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( @@ -143,9 +143,9 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) final List locks = toolbox.getTaskLockbox().findLocksForTask(task); if (locks.get(0).getGranularity() == LockGranularity.SEGMENT) { - for (Entry> entry : intervalToPartitionIds.entrySet()) { + for (Entry> entry : intervalToPartitionIds.entrySet()) { final Interval interval = entry.getKey(); - final List partitionIds = entry.getValue(); + final Set partitionIds = entry.getValue(); partitionIds.forEach(partitionId -> toolbox.getTaskLockbox().unlock(task, interval, partitionId)); } } 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 index c4429c587975..60d9264bb2a7 100644 --- 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 @@ -96,12 +96,12 @@ public static boolean isLockCoversSegments( final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; return timeChunkLock.getInterval().contains(segment.getInterval()) && timeChunkLock.getDataSource().equals(segment.getDataSource()) - && timeChunkLock.getVersion().compareTo(segment.getVersion()) >= 0; + && timeChunkLock.getVersion().compareTo(segment.getMajorVersion()) >= 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.getVersion().compareTo(segment.getMajorVersion()) >= 0 && segmentLock.getPartitionId() == segment.getShardSpec().getPartitionNum(); } } 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 f99b2dfc8a29..4a30bc80a8d1 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 @@ -26,9 +26,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction; import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; @@ -38,6 +40,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.timeline.DataSegment; @@ -323,8 +326,6 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i throws IOException { if (requireLockInputSegments()) { - // TODO: check changeSegmentGranularity and get timeChunkLock here - // This method finds segments falling in all given intervals and then tries to lock those segments. // Thus, there might be a race between calling findInputSegments() and tryLockWithSegments(), // i.e., a new segment can be added to the interval or an existing segment might be removed. @@ -343,16 +344,15 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } - boolean tryLockWithSegments(TaskActionClient client, List segments) - throws IOException + boolean tryLockWithSegments(TaskActionClient client, List segments) throws IOException { - if (requireLockInputSegments()) { - if (segments.isEmpty()) { - changeSegmentGranularity = false; - overwritingRootGenPartitions = Collections.emptyMap(); - return true; - } + if (segments.isEmpty()) { + changeSegmentGranularity = false; + overwritingRootGenPartitions = Collections.emptyMap(); + return true; + } + if (requireLockInputSegments()) { // Create a timeline to find latest segments only final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); @@ -397,6 +397,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); } intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); + final Closer lockCloserOnError = Closer.create(); for (Entry> entry : intervalToSegments.entrySet()) { final Interval interval = entry.getKey(); final Set partitionIds = entry.getValue().stream() @@ -406,12 +407,20 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) new SegmentLockTryAcquireAction( TaskLockType.EXCLUSIVE, interval, - entry.getValue().get(0).getVersion(), + entry.getValue().get(0).getMajorVersion(), partitionIds ) ); + + lockResults.stream() + .filter(LockResult::isOk) + .map(result -> (SegmentLock) result.getTaskLock()) + .forEach(segmentLock -> lockCloserOnError.register(() -> client.submit( + new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) + ))); + if (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk())) { - // TODO: unlock + lockCloserOnError.close(); return false; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java index b4e3dd55d7b1..a0097fa00794 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java @@ -85,11 +85,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { - if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { + if (unusedSegment.getMajorVersion().compareTo(myLock.getVersion()) > 0) { throw new ISE( "WTF?! Unused segment[%s] has version[%s] > task version[%s]", unusedSegment.getId(), - unusedSegment.getVersion(), + unusedSegment.getMajorVersion(), myLock.getVersion() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java index 05a96f5a3849..8b7e12b092a7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java @@ -36,7 +36,6 @@ import java.util.List; import java.util.Map; -// TODO: caching?? public abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocator { private final TaskToolbox toolbox; @@ -110,8 +109,7 @@ public String getSequenceName(Interval interval, InputRow inputRow) } /** - * Create a sequence name from the given shardSpec and interval. The shardSpec must be the original one before calling - * {@link #makeShardSpec(ShardSpec, int)} to apply the proper partitioning. + * Create a sequence name from the given shardSpec and interval. * * See {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec} as an example of partitioning. */ 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 b93104769fe1..8014432c3dab 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 @@ -301,7 +301,7 @@ public List findInputSegments(TaskActionClient taskActionClient, Li public boolean changeSegmentGranularity(List intervalOfExistingSegments) { return (keepSegmentGranularity != null && !keepSegmentGranularity) - || (segmentGranularity != null); // TODO: check segmentGranularity is different + || (segmentGranularity != null); } @Override 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 d3bdc82107dd..210b0fefb7f3 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 @@ -1171,7 +1171,6 @@ ShardSpec getShardSpec(Interval interval, InputRow row) if (shardSpecs == null || shardSpecs.isEmpty()) { throw new ISE("Failed to get shardSpec for interval[%s]", interval); } - // TODO: cache shardSpecLookup return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(row.getTimestampFromEpoch(), row); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java index 3cd87c627ad8..4cdabb0446b0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java @@ -22,7 +22,9 @@ import org.apache.druid.indexing.common.task.IndexTask.SequenceNameGenerator; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; -// TODO: rename??? -public interface IndexTaskSegmentAllocator extends SegmentAllocator, SequenceNameGenerator +/** + * Segment allocator interface for {@link IndexTask} + */ +interface IndexTaskSegmentAllocator extends SegmentAllocator, SequenceNameGenerator { } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java index 432aa50d4ae4..12f117b94557 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java @@ -93,11 +93,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { - if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { + if (unusedSegment.getMajorVersion().compareTo(myLock.getVersion()) > 0) { throw new ISE( "WTF?! Unused segment[%s] has version[%s] > task version[%s]", unusedSegment.getId(), - unusedSegment.getVersion(), + unusedSegment.getMajorVersion(), myLock.getVersion() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index c184bede2cdf..2d89589b1fb9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -242,7 +242,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception public void announceSegment(final DataSegment segment) throws IOException { // Side effect: Calling announceSegment causes a lock to be acquired - // TODO: get lock before allocating segment Preconditions.checkNotNull( toolbox.getTaskActionClient().submit( new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, segment.getInterval(), lockTimeoutMs) @@ -268,7 +267,6 @@ public void unannounceSegment(final DataSegment segment) throws IOException public void announceSegments(Iterable segments) throws IOException { // Side effect: Calling announceSegments causes locks to be acquired - // TODO: get lock before allocating segment for (DataSegment segment : segments) { Preconditions.checkNotNull( toolbox.getTaskActionClient().submit( @@ -308,7 +306,6 @@ public String getVersion(final Interval interval) { try { // Side effect: Calling getVersion causes a lock to be acquired - // TODO: get lock to get the version? final TimeChunkLockAcquireAction action = new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs); final TaskLock lock = Preconditions.checkNotNull( toolbox.getTaskActionClient().submit(action), @@ -515,7 +512,7 @@ public boolean changeSegmentGranularity(List intervalOfExistingSegment @Override public Granularity getSegmentGranularity(Interval interval) { - return null; + return spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java index d77a685a29ec..340b3cd050f9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java @@ -86,11 +86,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { - if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { + if (unusedSegment.getMajorVersion().compareTo(myLock.getVersion()) > 0) { throw new ISE( "WTF?! Unused segment[%s] has version[%s] > task version[%s]", unusedSegment.getId(), - unusedSegment.getVersion(), + unusedSegment.getMajorVersion(), myLock.getVersion() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index f435ed4817d1..a7640c267256 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -284,7 +284,6 @@ public SegmentIdWithShardSpec allocate( private SegmentAllocator createSegmentAllocator() { - // TODO: what if intervals are missing? if (ingestionSchema.getIOConfig().isAppendToExisting() || !isChangeSegmentGranularity()) { return new ActionBasedSegmentAllocator( toolbox.getTaskActionClient(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 4d1e0bf4e8bc..1e24a6687364 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -351,7 +351,7 @@ private List> getTimelineForSegmentIds for (Interval interval : windowedSegmentId.getIntervals()) { final TimelineObjectHolder existingHolder = timeline.get(interval); if (existingHolder != null) { - if (!existingHolder.getVersion().equals(segment.getVersion())) { + if (!existingHolder.getVersion().equals(segment.getMajorVersion())) { throw new ISE("Timeline segments with the same interval should have the same version: " + "existing version[%s] vs new segment[%s]", existingHolder.getVersion(), segment); } @@ -360,7 +360,7 @@ private List> getTimelineForSegmentIds timeline.put(interval, new TimelineObjectHolder<>( interval, segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), new PartitionHolder(segment.getShardSpec().createChunk(segment)) )); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 88e81675fc3e..5c32fe273f4b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -532,63 +532,34 @@ private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment reques * The given action should be finished as soon as possible because all other methods in this class are blocked until * this method is finished. * - * @param task task performing a critical action - * @param intervalToPartitionIds partitionIds which should be locked by task - * @param action action to be performed inside of the critical section + * @param task task performing a critical action + * @param intervals intervals + * @param action action to be performed inside of the critical section */ - public T doInCriticalSection( - Task task, - Map> intervalToPartitionIds, - CriticalAction action - ) throws Exception + public T doInCriticalSection(Task task, List intervals, CriticalAction action) throws Exception { giant.lock(); - // TODO: reduce contention by checking dataSource and interval. - // TODO: also cache taskLocks - try { - return action.perform(isTaskLocksValid(task, intervalToPartitionIds)); + return action.perform(isTaskLocksValid(task, intervals)); } finally { giant.unlock(); } } - private boolean isTaskLocksValid(Task task, Map> intervalToPartitionIds) + private boolean isTaskLocksValid(Task task, List intervals) { giant.lock(); try { - return intervalToPartitionIds - .entrySet() + return intervals .stream() - .allMatch(entry -> { - // TODO: segment lock validation - // needs to check used segments, valid segment locks for used segments, - final List lockPosses = getOnlyTaskLockPosseContainingInterval( - task, - entry.getKey(), - entry.getValue() - ); + .allMatch(interval -> { + final List lockPosses = getOnlyTaskLockPosseContainingInterval(task, interval); // Tasks cannot enter the critical section with a shared lock - final boolean allLocksAreValid = lockPosses.stream().allMatch( - posse -> !posse.getTaskLock().isRevoked() && posse.getTaskLock().getLockType() != TaskLockType.SHARED + return lockPosses.stream().map(TaskLockPosse::getTaskLock).allMatch( + lock -> !lock.isRevoked() && lock.getLockType() != TaskLockType.SHARED ); - - final Set remainingPartitionIds = new HashSet<>(entry.getValue()); - if (allLocksAreValid) { - for (TaskLockPosse lockPosse : lockPosses) { - if (lockPosse.getTaskLock().getGranularity() == LockGranularity.TIME_CHUNK) { - return true; - } else { - final SegmentLock segmentLock = (SegmentLock) lockPosse.getTaskLock(); - remainingPartitionIds.remove(segmentLock.getPartitionId()); - } - } - return remainingPartitionIds.isEmpty(); - } else { - return false; - } }); } finally { @@ -911,7 +882,7 @@ List getOnlyTaskLockPosseContainingInterval(Task task, Interval i { giant.lock(); try { - return getOnlyTaskLockPosseContainingInterval(task, interval, Collections.emptyList()); + return getOnlyTaskLockPosseContainingInterval(task, interval, Collections.emptySet()); } finally { giant.unlock(); @@ -919,7 +890,7 @@ List getOnlyTaskLockPosseContainingInterval(Task task, Interval i } @VisibleForTesting - List getOnlyTaskLockPosseContainingInterval(Task task, Interval interval, List partitionIds) + List getOnlyTaskLockPosseContainingInterval(Task task, Interval interval, Set partitionIds) { giant.lock(); try { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java index f8f04663826d..9fc905c278e5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java @@ -105,7 +105,7 @@ public void testSimple() throws Exception acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); actionTestKit.getTaskLockbox().doInCriticalSection( task, - Collections.singletonMap(INTERVAL, Collections.emptyList()), + Collections.singletonList(INTERVAL), CriticalAction.builder() .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) .onInvalidLocks( @@ -138,7 +138,7 @@ public void testFailBadVersion() throws Exception thrown.expectMessage(CoreMatchers.containsString("are not covered by locks")); final Set segments = actionTestKit.getTaskLockbox().doInCriticalSection( task, - Collections.singletonMap(INTERVAL, Collections.emptyList()), + Collections.singletonList(INTERVAL), CriticalAction.>builder() .onValidLocks(() -> action.perform(task, actionTestKit.getTaskActionToolbox())) .onInvalidLocks( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index e28ca22f7c99..ea6b7cc637f5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -379,7 +379,7 @@ public void testBasics() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -441,7 +441,7 @@ public void testLateData() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -506,7 +506,7 @@ public void testMaxRowsPerSegment() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -573,7 +573,7 @@ public void testMaxTotalRows() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -647,7 +647,7 @@ public void testTransformSpec() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -777,7 +777,7 @@ public void testNoReportParseExceptions() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -880,7 +880,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -1076,7 +1076,7 @@ public void testRestore() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -1159,7 +1159,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 3bb1fc0ae2cc..812a6c3147c9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -364,7 +364,7 @@ public void testBasics() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -440,7 +440,7 @@ public void testTransformSpec() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -556,7 +556,7 @@ public void testNoReportParseExceptions() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -649,7 +649,7 @@ public void testRestore() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -744,7 +744,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getVersion(), + publishedSegment.getMajorVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index d535aae6e6d5..761dee61efda 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLockType; @@ -121,7 +120,7 @@ public void testDoInCriticalSectionWithDifferentTasks() return lockbox.doInCriticalSection( lowPriorityTask, - Collections.singletonMap(interval, Collections.emptyList()), + Collections.singletonList(interval), CriticalAction.builder() .onValidLocks( () -> { @@ -151,7 +150,7 @@ public void testDoInCriticalSectionWithDifferentTasks() return lockbox.doInCriticalSection( highPriorityTask, - Collections.singletonMap(interval, Collections.emptyList()), + Collections.singletonList(interval), CriticalAction.builder() .onValidLocks( () -> { @@ -201,7 +200,7 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception final Future future1 = service.submit(() -> lockbox.doInCriticalSection( task, - ImmutableMap.of(intervals.get(0), Collections.emptyList(), intervals.get(1), Collections.emptyList()), + intervals.subList(0, 2), CriticalAction.builder() .onValidLocks( () -> { @@ -224,7 +223,7 @@ public void testDoInCriticalSectionWithOverlappedIntervals() throws Exception latch.await(); return lockbox.doInCriticalSection( task, - ImmutableMap.of(intervals.get(1), Collections.emptyList(), intervals.get(2), Collections.emptyList()), + intervals.subList(1, 3), CriticalAction.builder() .onValidLocks( () -> { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index d810b8bacf30..4ef6c1d40a57 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -494,7 +494,7 @@ public void testDoInCriticalSectionWithSharedLock() throws Exception Assert.assertFalse( lockbox.doInCriticalSection( task, - Collections.singletonMap(interval, Collections.emptyList()), + Collections.singletonList(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -512,7 +512,7 @@ public void testDoInCriticalSectionWithExclusiveLock() throws Exception Assert.assertTrue( lockbox.doInCriticalSection( task, - Collections.singletonMap(interval, Collections.emptyList()), + Collections.singletonList(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -531,7 +531,7 @@ public void testDoInCriticalSectionWithSmallerInterval() throws Exception Assert.assertTrue( lockbox.doInCriticalSection( task, - Collections.singletonMap(interval, Collections.emptyList()), + Collections.singletonList(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -557,7 +557,7 @@ public void testPreemptionAndDoInCriticalSection() throws Exception Assert.assertTrue( lockbox.doInCriticalSection( highPriorityTask, - Collections.singletonMap(interval, Collections.emptyList()), + Collections.singletonList(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); @@ -582,7 +582,7 @@ public void testDoInCriticalSectionWithRevokedLock() throws Exception Assert.assertFalse( lockbox.doInCriticalSection( lowPriorityTask, - Collections.singletonMap(interval, Collections.emptyList()), + Collections.singletonList(interval), CriticalAction.builder().onValidLocks(() -> true).onInvalidLocks(() -> false).build() ) ); diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java index 5fe3dab05582..2cded45e6e4f 100644 --- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java @@ -212,6 +212,12 @@ public int getEndRootPartitionId() return endRootPartitionId; } + @Override + public String getMajorVersion() + { + return baseSegment.getId().getVersion(); + } + @Override public short getMinorVersion() { diff --git a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java index cd7220e6c196..6128c7517272 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/QueryRunnerTestHelper.java @@ -454,7 +454,6 @@ public Sequence run(QueryPlus queryPlus, Map responseConte segments.addAll(timeline.lookup(interval)); } List> sequences = new ArrayList<>(); - // TODO: use other segment instead of ReferenceCountingSegment for (TimelineObjectHolder holder : toolChest.filterSegments(query, segments)) { Segment segment = holder.getObject().getChunk(0).getObject(); QueryPlus queryPlusRunning = queryPlus.withQuerySegmentSpec( diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java index 5475ac3dc573..4ceee6d71807 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java @@ -483,7 +483,7 @@ private QueryableIndex makeAppendedMMappedIndex( ShardSpec noneShardSpec = NoneShardSpec.instance(); for (int i = 0; i < intervals.size(); i++) { - timeline.add(intervals.get(i), i, noneShardSpec.createChunk(new OvershadowableFile(filesToMap.get(i)))); + timeline.add(intervals.get(i), i, noneShardSpec.createChunk(new OvershadowableFile(i, filesToMap.get(i)))); } final List adapters = Lists.newArrayList( @@ -576,10 +576,12 @@ public QueryableIndex apply(@Nullable File input) private static class OvershadowableFile implements Overshadowable { + private final String majorVersion; private final File file; - OvershadowableFile(File file) + OvershadowableFile(int majorVersion, File file) { + this.majorVersion = Integer.toString(majorVersion); this.file = file; } @@ -601,6 +603,12 @@ public int getEndRootPartitionId() return 0; } + @Override + public String getMajorVersion() + { + return majorVersion; + } + @Override public short getMinorVersion() { diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 7df034a0b6e9..91e34da2f67a 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -241,7 +241,7 @@ private void serverAddedSegment(final DruidServerMetadata server, final DataSegm timelines.put(segment.getDataSource(), timeline); } - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); + timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(selector)); selectors.put(segmentId, selector); } @@ -284,14 +284,14 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen selectors.remove(segmentId); final PartitionChunk removedPartition = timeline.remove( - segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector) + segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(selector) ); if (removedPartition == null) { log.warn( "Asked to remove timeline entry[interval: %s, version: %s] that doesn't exist", segment.getInterval(), - segment.getVersion() + segment.getMajorVersion() ); } else { runTimelineCallbacks(callback -> callback.segmentRemoved(segment)); diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java index 1f1d801d9e36..fba090c68609 100644 --- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java +++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java @@ -147,7 +147,7 @@ private void serverAddedSegment(final DruidServerMetadata server, final DataSegm timeline.add( segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), segment.getShardSpec().createChunk(segmentLoadInfo) ); segmentLoadInfos.put(segmentId, segmentLoadInfo); @@ -174,7 +174,7 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen segmentLoadInfos.remove(segmentId); final PartitionChunk removedPartition = timeline.remove( - segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk( + segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk( new SegmentLoadInfo( segment ) @@ -185,7 +185,7 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen log.warn( "Asked to remove timeline entry[interval: %s, version: %s] that doesn't exist", segment.getInterval(), - segment.getVersion() + segment.getMajorVersion() ); } } diff --git a/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java b/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java index 83fd298485b3..ec4f172ac094 100644 --- a/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java +++ b/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java @@ -114,6 +114,12 @@ public int getEndRootPartitionId() return segment.getEndRootPartitionId(); } + @Override + public String getMajorVersion() + { + return segment.getMajorVersion(); + } + @Override public short getMinorVersion() { diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java index 0d1f9ba3a4ec..0e4d3c3cb59a 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java @@ -191,6 +191,12 @@ public int getEndRootPartitionId() return segment.get().getEndRootPartitionId(); } + @Override + public String getMajorVersion() + { + return segment.get().getMajorVersion(); + } + @Override public short getMinorVersion() { diff --git a/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java b/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java index 71c2cadb105b..cb8e68f01293 100644 --- a/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java +++ b/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java @@ -74,7 +74,7 @@ public Void withHandle(Handle handle) throws Exception .put("start", segment.getInterval().getStart().toString()) .put("end", segment.getInterval().getEnd().toString()) .put("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .put("version", segment.getVersion()) + .put("version", segment.getMajorVersion()) .put("used", true) .put("payload", mapper.writeValueAsBytes(segment)) .build() diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index c04021c0e503..20b146bce1be 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -788,7 +788,7 @@ private boolean announceHistoricalSegment( .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getVersion()) + .bind("version", segment.getMajorVersion()) .bind("used", used) .bind("payload", jsonMapper.writeValueAsBytes(segment)) .execute(); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java index 6c33396de69f..9f155679de72 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java @@ -71,7 +71,7 @@ public void publishSegment(final DataSegment segment) throws IOException segment.getInterval().getStart().toString(), segment.getInterval().getEnd().toString(), (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, - segment.getVersion(), + segment.getMajorVersion(), true, jsonMapper.writeValueAsBytes(segment) ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index f8c72f18c5b8..1a431ca41c4e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -405,7 +405,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) sinks.put(identifier, retVal); metrics.setSinkCount(sinks.size()); - sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), identifier.getShardSpec().createChunk(retVal)); + sinkTimeline.add(retVal.getInterval(), retVal.getMajorVersion(), identifier.getShardSpec().createChunk(retVal)); } return retVal; @@ -1069,7 +1069,7 @@ private Object bootstrapSinksFromDisk() sinks.put(identifier, currSink); sinkTimeline.add( currSink.getInterval(), - currSink.getVersion(), + currSink.getMajorVersion(), identifier.getShardSpec().createChunk(currSink) ); @@ -1163,7 +1163,7 @@ public Void apply(@Nullable Object input) droppingSinks.remove(identifier); sinkTimeline.remove( sink.getInterval(), - sink.getVersion(), + sink.getMajorVersion(), identifier.getShardSpec().createChunk(sink) ); for (FireHydrant hydrant : sink) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java index 6e85794e69d1..a786763b5769 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java @@ -134,7 +134,7 @@ public static SegmentIdWithShardSpec fromDataSegment(final DataSegment segment) return new SegmentIdWithShardSpec( segment.getDataSource(), segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), segment.getShardSpec() ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java index bbee720a88ce..122d0fe4f9c3 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java @@ -141,7 +141,7 @@ static boolean isHandOffComplete(List serverView, Segm if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval()) && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() - && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 + && segmentLoadInfo.getSegment().getMajorVersion().compareTo(descriptor.getVersion()) >= 0 && segmentLoadInfo.getServers().stream().anyMatch(DruidServerMetadata::segmentReplicatable)) { return true; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index 005f167a1830..6385dafcf473 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -488,7 +488,7 @@ public void doRun() } ); handoffNotifier.registerSegmentHandoffCallback( - new SegmentDescriptor(sink.getInterval(), sink.getVersion(), config.getShardSpec().getPartitionNum()), + new SegmentDescriptor(sink.getInterval(), sink.getMajorVersion(), config.getShardSpec().getPartitionNum()), mergeExecutor, new Runnable() { @Override @@ -736,7 +736,7 @@ private void addSink(final Sink sink) metrics.setSinkCount(sinks.size()); sinkTimeline.add( sink.getInterval(), - sink.getVersion(), + sink.getMajorVersion(), new SingleElementPartitionChunk<>(sink) ); try { @@ -880,7 +880,7 @@ protected void abandonSegment(final long truncatedTime, final Sink sink) metrics.setSinkCount(sinks.size()); sinkTimeline.remove( sink.getInterval(), - sink.getVersion(), + sink.getMajorVersion(), new SingleElementPartitionChunk<>(sink) ); for (FireHydrant hydrant : sink) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index 67506d7b5c1e..a73535019cce 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -142,11 +142,6 @@ public void clearDedupCache() dedupSet.clear(); } - public String getVersion() - { - return version; - } - public Interval getInterval() { return interval; @@ -429,6 +424,12 @@ public int getEndRootPartitionId() return shardSpec.getEndRootPartitionId(); } + @Override + public String getMajorVersion() + { + return version; + } + @Override public short getMinorVersion() { diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index 5fde55d0c274..6417822b79c9 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -168,7 +168,7 @@ public boolean loadSegment(final DataSegment segment) throws SegmentLoadingExcep dataSourceState.getTimeline(); final PartitionHolder entry = loadedIntervals.findEntry( segment.getInterval(), - segment.getVersion() + segment.getMajorVersion() ); if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { @@ -178,7 +178,7 @@ public boolean loadSegment(final DataSegment segment) throws SegmentLoadingExcep final ReferenceCountingSegment referenceCountingSegment = new ReferenceCountingSegment(adapter); loadedIntervals.add( segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), segment.getShardSpec().createChunk(referenceCountingSegment) ); dataSourceState.addSegment(segment); @@ -225,7 +225,7 @@ public void dropSegment(final DataSegment segment) final ShardSpec shardSpec = segment.getShardSpec(); final PartitionChunk removed = loadedIntervals.remove( segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), // remove() internally searches for a partitionChunk to remove which is *equal* to the given // partitionChunk. Note that partitionChunk.equals() checks only the partitionNum, but not the object. segment.getShardSpec().createChunk( @@ -250,7 +250,7 @@ public void dropSegment(final DataSegment segment) "Told to delete a queryable on dataSource[%s] for interval[%s] and version[%s] that I don't have.", dataSourceName, segment.getInterval(), - segment.getVersion() + segment.getMajorVersion() ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index 655d6bdf22ff..85fc20af45a4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -350,7 +350,7 @@ public Builder withDataSources(Collection dataSourcesC dataSource.getSegments().forEach( segment -> timeline.add( segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), segment.getShardSpec().createChunk(segment) ) ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index 660ec7029ae6..3fdac3a30805 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -73,7 +73,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) //Remove all segments in db that are overshadowed by served segments for (DataSegment dataSegment : params.getAvailableSegments()) { VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion(), dataSegment)) { + if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getMajorVersion(), dataSegment)) { coordinator.removeSegment(dataSegment); stats.addToGlobalStat("overShadowedCount", 1); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index e82aba52e0d2..20aee98d4a56 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -145,7 +145,7 @@ private Set determineOvershadowedSegments(DruidCoordinatorRuntimePa for (DataSegment segment : params.getAvailableSegments()) { timelines .computeIfAbsent(segment.getDataSource(), dataSource -> new VersionedIntervalTimeline<>(Ordering.natural())) - .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + .add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); } Set overshadowed = new HashSet<>(); diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 05535053f83a..1fd3e98273e0 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -691,7 +691,7 @@ static boolean isSegmentLoaded(Iterable serverView, Se for (ImmutableSegmentLoadInfo segmentLoadInfo : serverView) { if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval()) && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() - && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 + && segmentLoadInfo.getSegment().getMajorVersion().compareTo(descriptor.getVersion()) >= 0 && Iterables.any( segmentLoadInfo.getServers(), DruidServerMetadata::segmentReplicatable )) { diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 7dd7847022f7..3704b1d77be7 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2831,9 +2831,9 @@ public Map getLoadSpec() @Override @JsonProperty - public String getVersion() + public String getMajorVersion() { - return baseSegment.getVersion(); + return baseSegment.getMajorVersion(); } @Override @@ -2939,7 +2939,7 @@ public boolean isOvershadow(DataSegment other) { if (getDataSource().equals(other.getDataSource()) && getInterval().overlaps(other.getInterval()) - && getVersion().equals(other.getVersion())) { + && getMajorVersion().equals(other.getMajorVersion())) { return getStartRootPartitionId() <= other.getStartRootPartitionId() && getEndRootPartitionId() >= other.getEndRootPartitionId() && getMinorVersion() > other.getMinorVersion(); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java index 4d7d1108dd91..44a32f9f1609 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java @@ -44,7 +44,7 @@ public Set findUsedSegments(Set identifiers for (DataSegment dataSegment : appenderatorTester.getPushedSegments()) { timeline.add( dataSegment.getInterval(), - dataSegment.getVersion(), + dataSegment.getMajorVersion(), dataSegment.getShardSpec().createChunk(dataSegment) ); } diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java index 14cbdbd90a4c..e6152b2798e8 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -402,7 +402,7 @@ private void assertResult(List expectedExistingSegments) throws Seg ); expectedTimeline.add( segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), segment.getShardSpec().createChunk(new ReferenceCountingSegment(segmentLoader.getSegment(segment))) ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index 66b73a789479..3e529e0eb788 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -94,13 +94,13 @@ public String compactSegments( segments.forEach( segment -> timeline.remove( segment.getInterval(), - segment.getVersion(), + segment.getMajorVersion(), segment.getShardSpec().createChunk(segment) ) ); timeline.add( compactInterval, - compactSegment.getVersion(), + compactSegment.getMajorVersion(), compactSegment.getShardSpec().createChunk(compactSegment) ); return "task_" + idSuffix++; @@ -135,18 +135,18 @@ public void setup() for (int j = 0; j < 4; j++) { for (int k = 0; k < 2; k++) { DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); } } for (int j = 7; j < 9; j++) { for (int k = 0; k < 2; k++) { DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); } } @@ -379,7 +379,7 @@ private void assertCompactSegments( Assert.assertEquals(1, chunks.size()); DataSegment segment = chunks.get(0).getObject(); Assert.assertEquals(expectedInterval, segment.getInterval()); - Assert.assertEquals(expectedVersionSupplier.get(), segment.getVersion()); + Assert.assertEquals(expectedVersionSupplier.get(), segment.getMajorVersion()); } } @@ -397,7 +397,7 @@ private void assertLastSegmentNotCompacted(DruidCoordinatorSegmentCompactor comp for (PartitionChunk chunk : chunks) { DataSegment segment = chunk.getObject(); Assert.assertEquals(interval, segment.getInterval()); - Assert.assertEquals("version", segment.getVersion()); + Assert.assertEquals("version", segment.getMajorVersion()); } } } @@ -427,13 +427,13 @@ private void addMoreData(String dataSource, int day) DataSegment newSegment = createSegment(dataSource, day, true, i); dataSources.get(dataSource).add( newSegment.getInterval(), - newSegment.getVersion(), + newSegment.getMajorVersion(), newSegment.getShardSpec().createChunk(newSegment) ); newSegment = createSegment(dataSource, day, false, i); dataSources.get(dataSource).add( newSegment.getInterval(), - newSegment.getVersion(), + newSegment.getMajorVersion(), newSegment.getShardSpec().createChunk(newSegment) ); } diff --git a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java index 7192c61a93a0..172c6d705954 100644 --- a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java @@ -276,6 +276,12 @@ public int getEndRootPartitionId() return partitionId + 1; } + @Override + public String getMajorVersion() + { + return "v1"; + } + @Override public short getMinorVersion() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 29f981619660..248911ffca1a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -266,7 +266,7 @@ public Enumerable scan(DataContext root) val.getInterval().getStart().toString(), val.getInterval().getEnd().toString(), val.getSize(), - val.getVersion(), + val.getMajorVersion(), Long.valueOf(val.getShardSpec().getPartitionNum()), numReplicas, numRows, @@ -299,7 +299,7 @@ public Enumerable scan(DataContext root) val.getKey().getInterval().getStart().toString(), val.getKey().getInterval().getEnd().toString(), val.getKey().getSize(), - val.getKey().getVersion(), + val.getKey().getMajorVersion(), Long.valueOf(val.getKey().getShardSpec().getPartitionNum()), numReplicas, val.getValue().getNumRows(), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 527f4acb0ce1..4e01ac05d9c4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -84,7 +84,7 @@ public SpecificSegmentsQuerySegmentWalker add( } final VersionedIntervalTimeline timeline = timelines.get(descriptor.getDataSource()); - timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(new ReferenceCountingSegment(segment))); + timeline.add(descriptor.getInterval(), descriptor.getMajorVersion(), descriptor.getShardSpec().createChunk(new ReferenceCountingSegment(segment))); segments.add(descriptor); closeables.add(index); return this; From 6a72b477ac46f315209f9714f7e06e163603d4ed Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 24 Apr 2019 22:59:16 -0700 Subject: [PATCH 11/52] fix teamcity --- .../common/actions/LocalTaskActionClient.java | 3 +-- .../druid/indexing/overlord/CriticalAction.java | 4 +--- .../druid/indexing/overlord/TaskLockbox.java | 16 ---------------- 3 files changed, 2 insertions(+), 21 deletions(-) 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 71442b12bcb2..9b2a7659dfb0 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 @@ -28,14 +28,13 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; public class LocalTaskActionClient implements TaskActionClient { private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); - private final ConcurrentMap, AtomicInteger> actionCountMap = new ConcurrentHashMap<>(); + private final ConcurrentHashMap, AtomicInteger> actionCountMap = new ConcurrentHashMap<>(); private final Task task; private final TaskStorage storage; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java index 1115b7398f8b..9a860ea681e0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.overlord; import com.google.common.base.Preconditions; -import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.Task; import org.joda.time.Interval; @@ -28,8 +27,7 @@ * This class represents a critical action must be done while the task's lock is guaranteed to not be revoked in the * middle of the action. * - * Implementations must not change the lock state by calling {@link TaskLockbox#lock(TaskLockType, Task, Interval)}, - * {@link TaskLockbox#lock(TaskLockType, Task, Interval, long)}, {@link TaskLockbox#tryTimeChunkLock(TaskLockType, Task, Interval)}, + * Implementations must not change the lock state by calling {@link TaskLockbox#lock)}, {@link TaskLockbox#tryLock)}, * or {@link TaskLockbox#unlock(Task, Interval)}. * * Also, implementations should be finished as soon as possible because all methods in {@link TaskLockbox} are blocked diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 5c32fe273f4b..6d170ba56b08 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -272,11 +272,6 @@ private TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLoc /** * Acquires a lock on behalf of a task. Blocks until the lock is acquired. * - * @param granularity lock granularity - * @param lockType lock type - * @param task task to acquire lock for - * @param interval interval to lock - * * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a * {@link LockResult#revoked} flag. * @@ -303,12 +298,6 @@ public LockResult lock(final Task task, final LockRequest request) throws Interr /** * Acquires a lock on behalf of a task, waiting up to the specified wait time if necessary. * - * @param granularity lock granularity - * @param lockType lock type - * @param task task to acquire a lock for - * @param interval interval to lock - * @param timeoutMs maximum time to wait - * * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a * {@link LockResult#revoked} flag. * @@ -337,11 +326,6 @@ public LockResult lock(final Task task, final LockRequest request, long timeoutM * Attempt to acquire a lock for a task, without removing it from the queue. Can safely be called multiple times on * the same task until the lock is preempted. * - * @param granularity lock granularity - * @param lockType type of lock to be acquired - * @param task task that wants a lock - * @param interval interval to lock - * * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a * {@link LockResult#revoked} flag. * From 6c5d477de50e9a999d3705e9010b4d834f1cffed Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 24 Apr 2019 23:41:35 -0700 Subject: [PATCH 12/52] remove unused class --- .../partition/NoneShardSpecFactory.java | 56 ------------------- .../partition/OvershadowableManager.java | 2 +- .../partition/OverwritingShardSpec.java | 4 +- .../timeline/partition/ShardSpecFactory.java | 1 - .../timeline/partition/NoneShardSpecTest.java | 7 --- 5 files changed, 4 insertions(+), 66 deletions(-) delete mode 100644 core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.java diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.java b/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.java deleted file mode 100644 index 27cca23cd3f6..000000000000 --- a/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpecFactory.java +++ /dev/null @@ -1,56 +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.timeline.partition; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import javax.annotation.Nullable; - -public class NoneShardSpecFactory implements ShardSpecFactory -{ - private static NoneShardSpecFactory INSTANCE = new NoneShardSpecFactory(); - - public static NoneShardSpecFactory instance() - { - return INSTANCE; - } - - private NoneShardSpecFactory() - { - } - - @Override - public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) - { - return NoneShardSpec.instance(); - } - - @Override - public ShardSpec create(ObjectMapper objectMapper, int partitionId) - { - return NoneShardSpec.instance(); - } - - @Override - public Class getShardSpecClass() - { - return NoneShardSpec.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 index 8be997849bf9..2b1a3e4703d0 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -66,7 +66,7 @@ private enum State private final Map> knownPartitionChunks; // served segments - // start partitionId -> end partitionId -> minorVersion -> atomicUpdateGroup + // (start partitionId, end partitionId) -> minorVersion -> atomicUpdateGroup private final TreeMap>> standbyGroups; private final TreeMap>> visibleGroup; private final TreeMap>> overshadowedGroups; diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java index 74577b688faa..57e74f19a6a8 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java @@ -19,7 +19,9 @@ package org.apache.druid.timeline.partition; -// TODO: rename +/** + * TODO + */ public interface OverwritingShardSpec extends ShardSpec { default OverwritingShardSpec withAtomicUpdateGroupSize(int atomicUpdateGroupSize) 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 index 5da2bd5f73b0..7c520c742232 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java @@ -27,7 +27,6 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes({ - @JsonSubTypes.Type(name = "none", value = NoneShardSpecFactory.class), @JsonSubTypes.Type(name = "numbered", value = NumberedShardSpecFactory.class), @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpecFactory.class), @JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwritingShardSpecFactory.class), diff --git a/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java index 6099bb92e09e..04a519dc5470 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/NoneShardSpecTest.java @@ -58,11 +58,4 @@ public void testPartitionFieldIgnored() throws IOException final ShardSpec noneShardSpec = mapper.readValue(jsonStr, ShardSpec.class); noneShardSpec.equals(NoneShardSpec.instance()); } - - @Test - public void testCompatible() - { - final NoneShardSpec s1 = NoneShardSpec.instance(); - Assert.assertTrue(s1.isCompatible(NoneShardSpecFactory.instance().getShardSpecClass())); - } } From fda329a7c7b037bea5aaa56e3bbdae8c76b15cb6 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 25 Apr 2019 09:41:07 -0700 Subject: [PATCH 13/52] fix single map --- .../partition/OvershadowableManager.java | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) 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 index 2b1a3e4703d0..7d77a23d5cf6 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -668,10 +668,25 @@ public int size() @Override public V put(final short key, final V value) { - final V existing = isEmpty() ? null : this.val; - this.key = key; - this.val = value; - return existing; + 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 From c8a2cf581242e436c33d46ef8f3114ba99366632 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 26 Apr 2019 15:07:59 -0700 Subject: [PATCH 14/52] resolving todos --- .../timeline/partition/AtomicUpdateGroup.java | 7 + ...c.java => NumberedOverwriteShardSpec.java} | 16 +- .../NumberedOverwritingShardSpecFactory.java | 6 +- .../timeline/partition/NumberedShardSpec.java | 2 +- ...ShardSpec.java => OverwriteShardSpec.java} | 12 +- .../druid/timeline/partition/ShardSpec.java | 2 +- .../common/task/AbstractBatchIndexTask.java | 356 ++++++++++++++++++ .../indexing/common/task/AbstractTask.java | 286 +------------- .../AppenderatorDriverRealtimeIndexTask.java | 29 -- .../indexing/common/task/ArchiveTask.java | 29 -- .../indexing/common/task/CompactionTask.java | 8 +- .../indexing/common/task/HadoopIndexTask.java | 6 + .../indexing/common/task/HadoopTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 34 +- .../druid/indexing/common/task/KillTask.java | 32 -- .../druid/indexing/common/task/MoveTask.java | 29 -- .../druid/indexing/common/task/NoopTask.java | 31 -- .../common/task/RealtimeIndexTask.java | 29 -- .../common/task/RemoteSegmentAllocator.java | 2 +- .../indexing/common/task/RestoreTask.java | 29 -- .../batch/parallel/ParallelIndexSubTask.java | 10 +- .../parallel/ParallelIndexSupervisorTask.java | 10 +- .../druid/indexing/overlord/TaskLockbox.java | 28 +- .../SeekableStreamIndexTask.java | 30 -- .../druid/indexing/common/TestTasks.java | 57 --- .../common/task/CompactionTaskRunTest.java | 14 +- .../indexing/common/task/HadoopTaskTest.java | 6 + .../indexing/overlord/RealtimeishTask.java | 28 -- .../SingleTaskBackgroundRunnerTest.java | 30 -- .../indexing/overlord/TaskLifecycleTest.java | 79 ---- .../indexing/overlord/TaskLockboxTest.java | 28 -- .../overlord/http/OverlordResourceTest.java | 28 -- .../TransactionalSegmentPublisher.java | 12 +- ...exerSQLMetadataStorageCoordinatorTest.java | 6 +- .../server/shard/NumberedShardSpecTest.java | 4 +- 35 files changed, 467 insertions(+), 850 deletions(-) rename core/src/main/java/org/apache/druid/timeline/partition/{NumberedOverwritingShardSpec.java => NumberedOverwriteShardSpec.java} (92%) rename core/src/main/java/org/apache/druid/timeline/partition/{OverwritingShardSpec.java => OverwriteShardSpec.java} (66%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java 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 index 2f1f3e28cb35..0216bacbb779 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -29,6 +29,13 @@ 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> implements Overshadowable> { // Perhaps it would be worth to store these in a map of (partitionId -> partitionChunk) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java similarity index 92% rename from core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java rename to core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java index f955e8f0f998..f36f4a6e046f 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java @@ -30,7 +30,7 @@ import java.util.Map; import java.util.Objects; -public class NumberedOverwritingShardSpec implements OverwritingShardSpec +public class NumberedOverwriteShardSpec implements OverwriteShardSpec { private final int partitionId; @@ -40,7 +40,7 @@ public class NumberedOverwritingShardSpec implements OverwritingShardSpec private final short atomicUpdateGroupSize; // number of segments in atomicUpdateGroup @JsonCreator - public NumberedOverwritingShardSpec( + public NumberedOverwriteShardSpec( @JsonProperty("partitionId") int partitionId, @JsonProperty("startRootPartitionId") int startRootPartitionId, @JsonProperty("endRootPartitionId") int endRootPartitionId, @@ -90,7 +90,7 @@ public NumberedOverwritingShardSpec( this.atomicUpdateGroupSize = atomicUpdateGroupSize; } - public NumberedOverwritingShardSpec( + public NumberedOverwriteShardSpec( int partitionId, int startRootPartitionId, int endRootPartitionId, @@ -107,9 +107,9 @@ public NumberedOverwritingShardSpec( } @Override - public OverwritingShardSpec withAtomicUpdateGroupSize(short atomicUpdateGroupSize) + public OverwriteShardSpec withAtomicUpdateGroupSize(short atomicUpdateGroupSize) { - return new NumberedOverwritingShardSpec( + return new NumberedOverwriteShardSpec( this.partitionId, this.startRootPartitionId, this.endRootPartitionId, @@ -186,7 +186,7 @@ public boolean possibleInDomain(Map> domain) @Override public boolean isCompatible(Class other) { - return other == NumberedOverwritingShardSpec.class || other == NumberedShardSpec.class; + return other == NumberedOverwriteShardSpec.class || other == NumberedShardSpec.class; } @Override @@ -198,7 +198,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - NumberedOverwritingShardSpec that = (NumberedOverwritingShardSpec) o; + NumberedOverwriteShardSpec that = (NumberedOverwriteShardSpec) o; return partitionId == that.partitionId && startRootPartitionId == that.startRootPartitionId && endRootPartitionId == that.endRootPartitionId && @@ -215,7 +215,7 @@ public int hashCode() @Override public String toString() { - return "NumberedOverwritingShardSpec{" + + return "NumberedOverwriteShardSpec{" + "partitionNum=" + partitionId + ", startRootPartitionId=" + startRootPartitionId + ", endRootPartitionId=" + endRootPartitionId + 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 index 07f4aa639242..2901ed5d1a32 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java @@ -64,7 +64,7 @@ public short getMinorVersion() @Override public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { - return new NumberedOverwritingShardSpec( + return new NumberedOverwriteShardSpec( specOfPreviousMaxPartitionId == null ? PartitionIds.NON_ROOT_GEN_START_PARTITION_ID : specOfPreviousMaxPartitionId.getPartitionNum() + 1, @@ -77,12 +77,12 @@ public ShardSpec create(ObjectMapper objectMapper, @Nullable ShardSpec specOfPre @Override public ShardSpec create(ObjectMapper objectMapper, int partitionId) { - return new NumberedOverwritingShardSpec(partitionId, startRootPartitionId, endRootPartitionId, minorVersion); + return new NumberedOverwriteShardSpec(partitionId, startRootPartitionId, endRootPartitionId, minorVersion); } @Override public Class getShardSpecClass() { - return NumberedOverwritingShardSpec.class; + return NumberedOverwriteShardSpec.class; } } 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 9bc68809b40d..d6f98d71751d 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 @@ -86,7 +86,7 @@ public boolean possibleInDomain(Map> domain) @Override public boolean isCompatible(Class other) { - return other == NumberedShardSpec.class || other == NumberedOverwritingShardSpec.class; + return other == NumberedShardSpec.class || other == NumberedOverwriteShardSpec.class; } @JsonProperty("partitions") diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java similarity index 66% rename from core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java rename to core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java index 57e74f19a6a8..0fea5646e311 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OverwritingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java @@ -20,14 +20,18 @@ package org.apache.druid.timeline.partition; /** - * TODO + * 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 interface OverwritingShardSpec extends ShardSpec +public interface OverwriteShardSpec extends ShardSpec { - default OverwritingShardSpec withAtomicUpdateGroupSize(int atomicUpdateGroupSize) + default OverwriteShardSpec withAtomicUpdateGroupSize(int atomicUpdateGroupSize) { return withAtomicUpdateGroupSize((short) atomicUpdateGroupSize); } - OverwritingShardSpec withAtomicUpdateGroupSize(short atomicUpdateGroupSize); + OverwriteShardSpec withAtomicUpdateGroupSize(short atomicUpdateGroupSize); } 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 e8b527a7588c..1adb26bb6319 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 @@ -38,7 +38,7 @@ @JsonSubTypes.Type(name = "linear", value = LinearShardSpec.class), @JsonSubTypes.Type(name = "numbered", value = NumberedShardSpec.class), @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class), - @JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwritingShardSpec.class) + @JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwriteShardSpec.class) }) public interface ShardSpec { 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 000000000000..7d6ec9d8730d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -0,0 +1,356 @@ +/* + * 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.indexing.common.SegmentLock; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction; +import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.overlord.LockResult; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.io.Closer; +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 javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +public abstract class AbstractBatchIndexTask extends AbstractTask +{ + @Nullable + private Map overwritingRootGenPartitions; + + @Nullable + private Boolean changeSegmentGranularity; + + public static class OverwritingRootGenerationPartitions + { + private final int startRootPartitionId; + private final int endRootPartitionId; + private final short maxMinorVersion; + + private OverwritingRootGenerationPartitions(int startRootPartitionId, int endRootPartitionId, short maxMinorVersion) + { + this.startRootPartitionId = startRootPartitionId; + this.endRootPartitionId = endRootPartitionId; + this.maxMinorVersion = maxMinorVersion; + } + + public int getStartRootPartitionId() + { + return startRootPartitionId; + } + + public int getEndRootPartitionId() + { + return endRootPartitionId; + } + + public short getMinorVersionForNewSegments() + { + return (short) (maxMinorVersion + 1); + } + } + + protected AbstractBatchIndexTask(String id, String dataSource, Map context) + { + super(id, dataSource, context); + } + + protected AbstractBatchIndexTask( + String id, + @Nullable String groupId, + @Nullable TaskResource taskResource, + String dataSource, + @Nullable Map context + ) + { + super(id, groupId, taskResource, dataSource, context); + } + + public abstract boolean requireLockInputSegments(); + + public abstract List findInputSegments(TaskActionClient taskActionClient, List intervals) + throws IOException; + + public abstract boolean changeSegmentGranularity(List intervalOfExistingSegments); + + public abstract boolean isPerfectRollup(); + + /** + * Returns the segmentGranularity for the given interval. Usually tasks are supposed to return its segmentGranularity + * if exists. The compactionTask can return different segmentGranularity depending on its configuration and the input + * interval. + * + * @return segmentGranularity or null if it doesn't support it. + */ + @Nullable + public abstract Granularity getSegmentGranularity(Interval interval); + + boolean tryLockWithIntervals(TaskActionClient client, Set intervals) + throws IOException + { + return tryLockWithIntervals(client, new ArrayList<>(intervals)); + } + + protected boolean tryLockWithIntervals(TaskActionClient client, List intervals) + throws IOException + { + if (requireLockInputSegments()) { + // This method finds segments falling in all given intervals and then tries to lock those segments. + // Thus, there might be a race between calling findInputSegments() and tryLockWithSegments(), + // 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 findInputSegments() call, it may missing those + // segments. This is usually fine, but if you want to avoid this, you should use timeChunk lock instead. + if (!intervals.isEmpty()) { + return tryLockWithSegments(client, findInputSegments(client, intervals)); + } else { + return true; + } + } else { + changeSegmentGranularity = false; + overwritingRootGenPartitions = Collections.emptyMap(); + return true; + } + } + + boolean tryLockWithSegments(TaskActionClient client, List segments) throws IOException + { + if (segments.isEmpty()) { + changeSegmentGranularity = false; + overwritingRootGenPartitions = Collections.emptyMap(); + return true; + } + + if (requireLockInputSegments()) { + // Create a timeline to find latest segments only + final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); + + changeSegmentGranularity = changeSegmentGranularity(intervals); + if (changeSegmentGranularity || isPerfectRollup()) { + overwritingRootGenPartitions = Collections.emptyMap(); + // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined + final Set uniqueIntervals = new HashSet<>(); + for (Interval interval : JodaUtils.condenseIntervals(intervals)) { + final Granularity segmentGranularity = getSegmentGranularity(interval); + 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; + } else { + final List segmentsToLock; + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + segments + ); + segmentsToLock = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) + .stream() + .map(TimelineObjectHolder::getObject) + .flatMap(partitionHolder -> StreamSupport.stream( + partitionHolder.spliterator(), + false + )) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); + + final Map> intervalToSegments = new HashMap<>(); + for (DataSegment segment : segmentsToLock) { + intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); + } + intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); + final Closer lockCloserOnError = Closer.create(); + for (Entry> entry : intervalToSegments.entrySet()) { + final Interval interval = entry.getKey(); + final Set partitionIds = entry.getValue().stream() + .map(s -> s.getShardSpec().getPartitionNum()) + .collect(Collectors.toSet()); + final List lockResults = client.submit( + new SegmentLockTryAcquireAction( + TaskLockType.EXCLUSIVE, + interval, + entry.getValue().get(0).getMajorVersion(), + partitionIds + ) + ); + + lockResults.stream() + .filter(LockResult::isOk) + .map(result -> (SegmentLock) result.getTaskLock()) + .forEach(segmentLock -> lockCloserOnError.register(() -> client.submit( + new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) + ))); + + if (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk())) { + lockCloserOnError.close(); + return false; + } + } + return true; + } + } else { + changeSegmentGranularity = false; + overwritingRootGenPartitions = Collections.emptyMap(); + return true; + } + } + + /** + * This method is called when the task overwrites existing segments with segment locks. It verifies the input segments + * can be locked together, so that output segments can overshadow existing ones properly. + *

+ * This method checks two things: + *

+ * - Are rootPartition range of inputSegments adjacent? Two rootPartition ranges are adjacent if they are consecutive. + * - All atomicUpdateGroups of inputSegments must be full. (See {@code AtomicUpdateGroup#isFull()}). + */ + private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments) + { + if (inputSegments.isEmpty()) { + return; + } + + final List sortedSegments = verifyInputSegmentsToOverwritePerInterval(inputSegments); + final Interval interval = sortedSegments.get(0).getInterval(); + final short prevMaxMinorVersion = (short) sortedSegments + .stream() + .mapToInt(DataSegment::getMinorVersion) + .max() + .orElseThrow(() -> new ISE("Empty inputSegments")); + + if (overwritingRootGenPartitions == null) { + overwritingRootGenPartitions = new HashMap<>(); + } + overwritingRootGenPartitions.put( + interval, + new OverwritingRootGenerationPartitions( + sortedSegments.get(0).getStartRootPartitionId(), + sortedSegments.get(sortedSegments.size() - 1).getEndRootPartitionId(), + prevMaxMinorVersion + ) + ); + } + + protected boolean isChangeSegmentGranularity() + { + return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); + } + + Map getAllOverwritingSegmentMeta() + { + Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); + return Collections.unmodifiableMap(overwritingRootGenPartitions); + } + + @Nullable + public OverwritingRootGenerationPartitions getOverwritingSegmentMeta(Interval interval) + { + Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); + return overwritingRootGenPartitions.get(interval); + } + + public boolean isOverwriteMode() + { + Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); + return !overwritingRootGenPartitions.isEmpty(); + } + + private static List verifyInputSegmentsToOverwritePerInterval(List inputSegments) + { + if (inputSegments.isEmpty()) { + return inputSegments; + } + + Preconditions.checkArgument( + inputSegments.stream().allMatch(segment -> segment.getInterval().equals(inputSegments.get(0).getInterval())) + ); + + final List sortedSegments = new ArrayList<>(inputSegments); + sortedSegments.sort((s1, s2) -> { + if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) { + return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId()); + } else { + return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId()); + } + }); + + short atomicUpdateGroupSize = 1; + // sanity check + for (int i = 0; i < sortedSegments.size() - 1; i++) { + final DataSegment curSegment = sortedSegments.get(i); + final DataSegment nextSegment = sortedSegments.get(i + 1); + if (curSegment.getStartRootPartitionId() == nextSegment.getStartRootPartitionId() + && curSegment.getEndRootPartitionId() == nextSegment.getEndRootPartitionId()) { + // Input segments should have the same or consecutive rootPartition range + if (curSegment.getMinorVersion() != nextSegment.getMinorVersion() + || curSegment.getAtomicUpdateGroupSize() != nextSegment.getAtomicUpdateGroupSize()) { + throw new ISE( + "segment[%s] and segment[%s] have the same rootPartitionRange, but different minorVersion or atomicUpdateGroupSize", + curSegment, + nextSegment + ); + } + atomicUpdateGroupSize++; + } else { + if (curSegment.getEndRootPartitionId() != nextSegment.getStartRootPartitionId()) { + throw new ISE("Can't compact segments of non-consecutive rootPartition range"); + } + if (atomicUpdateGroupSize != curSegment.getAtomicUpdateGroupSize()) { + throw new ISE("All atomicUpdateGroup must be compacted together"); + } + atomicUpdateGroupSize = 1; + } + } + if (atomicUpdateGroupSize == sortedSegments.get(sortedSegments.size() - 1).getAtomicUpdateGroupSize()) { + throw new ISE("All atomicUpdateGroup must be compacted together"); + } + + return sortedSegments; + } +} 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 4a30bc80a8d1..370b30ed1ec8 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 @@ -24,43 +24,22 @@ import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.actions.LockListAction; -import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction; -import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; -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 javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; public abstract class AbstractTask implements Task { @@ -80,41 +59,6 @@ public abstract class AbstractTask implements Task private final Map context; - @Nullable - private Map overwritingRootGenPartitions; - - @Nullable - private Boolean changeSegmentGranularity; - - public static class OverwritingRootGenerationPartitions - { - private final int startRootPartitionId; - private final int endRootPartitionId; - private final short maxMinorVersion; - - private OverwritingRootGenerationPartitions(int startRootPartitionId, int endRootPartitionId, short maxMinorVersion) - { - this.startRootPartitionId = startRootPartitionId; - this.endRootPartitionId = endRootPartitionId; - this.maxMinorVersion = maxMinorVersion; - } - - public int getStartRootPartitionId() - { - return startRootPartitionId; - } - - public int getEndRootPartitionId() - { - return endRootPartitionId; - } - - public short getMinorVersionForNewSegments() - { - return (short) (maxMinorVersion + 1); - } - } - protected AbstractTask(String id, String dataSource, Map context) { this(id, null, null, dataSource, context); @@ -244,7 +188,7 @@ static String joinId(List objects) return ID_JOINER.join(objects); } - static String joinId(Object...objects) + static String joinId(Object... objects) { return ID_JOINER.join(objects); } @@ -298,232 +242,4 @@ public Map getContext() { return context; } - - public abstract boolean requireLockInputSegments(); - - public abstract List findInputSegments(TaskActionClient taskActionClient, List intervals) - throws IOException; - - public abstract boolean changeSegmentGranularity(List intervalOfExistingSegments); - - /** - * Returns the segmentGranularity for the given interval. Usually tasks are supposed to return its segmentGranularity - * if exists. The compactionTask can return different segmentGranularity depending on its configuration and the input - * interval. - * - * @return segmentGranularity or null if it doesn't support it. - */ - @Nullable - public abstract Granularity getSegmentGranularity(Interval interval); - - boolean tryLockWithIntervals(TaskActionClient client, Set intervals) - throws IOException - { - return tryLockWithIntervals(client, new ArrayList<>(intervals)); - } - - protected boolean tryLockWithIntervals(TaskActionClient client, List intervals) - throws IOException - { - if (requireLockInputSegments()) { - // This method finds segments falling in all given intervals and then tries to lock those segments. - // Thus, there might be a race between calling findInputSegments() and tryLockWithSegments(), - // 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 findInputSegments() call, it may missing those - // segments. This is usually fine, but if you want to avoid this, you should use timeChunk lock instead. - if (!intervals.isEmpty()) { - return tryLockWithSegments(client, findInputSegments(client, intervals)); - } else { - return true; - } - } else { - changeSegmentGranularity = false; - overwritingRootGenPartitions = Collections.emptyMap(); - return true; - } - } - - boolean tryLockWithSegments(TaskActionClient client, List segments) throws IOException - { - if (segments.isEmpty()) { - changeSegmentGranularity = false; - overwritingRootGenPartitions = Collections.emptyMap(); - return true; - } - - if (requireLockInputSegments()) { - // Create a timeline to find latest segments only - final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); - - changeSegmentGranularity = changeSegmentGranularity(intervals); - if (changeSegmentGranularity) { - overwritingRootGenPartitions = Collections.emptyMap(); - // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined - final Set uniqueIntervals = new HashSet<>(); - for (Interval interval : JodaUtils.condenseIntervals(intervals)) { - final Granularity segmentGranularity = getSegmentGranularity(interval); - 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; - } else { - final List segmentsToLock; - final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( - segments - ); - segmentsToLock = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) - .stream() - .map(TimelineObjectHolder::getObject) - .flatMap(partitionHolder -> StreamSupport.stream( - partitionHolder.spliterator(), - false - )) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); - - final Map> intervalToSegments = new HashMap<>(); - for (DataSegment segment : segmentsToLock) { - intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); - } - intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); - final Closer lockCloserOnError = Closer.create(); - for (Entry> entry : intervalToSegments.entrySet()) { - final Interval interval = entry.getKey(); - final Set partitionIds = entry.getValue().stream() - .map(s -> s.getShardSpec().getPartitionNum()) - .collect(Collectors.toSet()); - final List lockResults = client.submit( - new SegmentLockTryAcquireAction( - TaskLockType.EXCLUSIVE, - interval, - entry.getValue().get(0).getMajorVersion(), - partitionIds - ) - ); - - lockResults.stream() - .filter(LockResult::isOk) - .map(result -> (SegmentLock) result.getTaskLock()) - .forEach(segmentLock -> lockCloserOnError.register(() -> client.submit( - new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) - ))); - - if (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk())) { - lockCloserOnError.close(); - return false; - } - } - return true; - } - } else { - changeSegmentGranularity = false; - overwritingRootGenPartitions = Collections.emptyMap(); - return true; - } - } - - /** - * TODO: description about what this method verifies - * TODO: similar check in taskLockbox?? - */ - private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments) - { - if (inputSegments.isEmpty()) { - return; - } - - Preconditions.checkArgument( - inputSegments.stream().allMatch(segment -> segment.getInterval().equals(inputSegments.get(0).getInterval())) - ); - final Interval interval = inputSegments.get(0).getInterval(); - - inputSegments.sort((s1, s2) -> { - if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) { - return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId()); - } else { - return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId()); - } - }); - - short atomicUpdateGroupSize = 1; - // sanity check - for (int i = 0; i < inputSegments.size() - 1; i++) { - final DataSegment curSegment = inputSegments.get(i); - final DataSegment nextSegment = inputSegments.get(i + 1); - if (curSegment.getStartRootPartitionId() == nextSegment.getStartRootPartitionId() - && curSegment.getEndRootPartitionId() == nextSegment.getEndRootPartitionId()) { - // Input segments should have the same or consecutive rootPartition range - if (curSegment.getMinorVersion() != nextSegment.getMinorVersion() - || curSegment.getAtomicUpdateGroupSize() != nextSegment.getAtomicUpdateGroupSize()) { - throw new ISE( - "segment[%s] and segment[%s] have the same rootPartitionRange, but different minorVersion or atomicUpdateGroupSize", - curSegment, - nextSegment - ); - } - atomicUpdateGroupSize++; - } else { - if (curSegment.getEndRootPartitionId() != nextSegment.getStartRootPartitionId()) { - throw new ISE("Can't compact segments of non-consecutive rootPartition range"); - } - if (atomicUpdateGroupSize != curSegment.getAtomicUpdateGroupSize()) { - throw new ISE("All atomicUpdateGroup must be compacted together"); - } - atomicUpdateGroupSize = 1; - } - } - - final short prevMaxMinorVersion = (short) inputSegments - .stream() - .mapToInt(DataSegment::getMinorVersion) - .max() - .orElseThrow(() -> new ISE("Empty inputSegments")); - - if (overwritingRootGenPartitions == null) { - overwritingRootGenPartitions = new HashMap<>(); - } - overwritingRootGenPartitions.put( - interval, - new OverwritingRootGenerationPartitions( - inputSegments.get(0).getStartRootPartitionId(), - inputSegments.get(inputSegments.size() - 1).getEndRootPartitionId(), - prevMaxMinorVersion - ) - ); - } - - protected boolean isChangeSegmentGranularity() - { - return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); - } - - Map getAllOverwritingSegmentMeta() - { - Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); - return Collections.unmodifiableMap(overwritingRootGenPartitions); - } - - @Nullable - public OverwritingRootGenerationPartitions getOverwritingSegmentMeta(Interval interval) - { - Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); - return overwritingRootGenPartitions.get(interval); - } - - public boolean isOverwriteMode() - { - Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); - return !overwritingRootGenPartitions.isEmpty(); - } } 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 eea46ea9bd3d..004a6fb5b7b6 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 @@ -62,7 +62,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.ListenableFutures; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -87,12 +86,9 @@ 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.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.utils.CircularBuffer; -import org.joda.time.Interval; -import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.GET; import javax.ws.rs.Path; @@ -420,31 +416,6 @@ public TaskStatus run(final TaskToolbox toolbox) return TaskStatus.success(getId()); } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public boolean canRestore() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java index a0097fa00794..8f03c658d428 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java @@ -27,15 +27,11 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -109,29 +105,4 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } - - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } } 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 8014432c3dab..1e256d69bfe6 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 @@ -102,7 +102,7 @@ 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"; @@ -304,6 +304,12 @@ public boolean changeSegmentGranularity(List intervalOfExistingSegment || (segmentGranularity != null); } + @Override + public boolean isPerfectRollup() + { + return tuningConfig != null && tuningConfig.isForceGuaranteedRollup(); + } + @Override public Granularity getSegmentGranularity(Interval interval) { 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 eb3b4ba1f18a..4d9ae1b0a12d 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 @@ -225,6 +225,12 @@ public boolean changeSegmentGranularity(List intervalOfExistingSegment return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); } + @Override + public boolean isPerfectRollup() + { + return true; + } + @Nullable @Override public Granularity getSegmentGranularity(Interval 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 768735d32e2c..f32ab28b38be 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.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 210b0fefb7f3..c5491e9fd767 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 @@ -86,9 +86,7 @@ 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.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpecFactory; import org.apache.druid.utils.CircularBuffer; @@ -121,7 +119,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -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(); @@ -280,6 +278,12 @@ public boolean changeSegmentGranularity(List intervalOfExistingSegment return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); } + @Override + public boolean isPerfectRollup() + { + return ingestionSchema.tuningConfig.isForceGuaranteedRollup(); + } + @Nullable @Override public Granularity getSegmentGranularity(Interval interval) @@ -711,30 +715,6 @@ private Pair createShardSpecFactoryForGuaranteedRollu return Pair.of(new HashBasedNumberedShardSpecFactory(partitionDimensions, numShards), numShards); } - private Pair createShardSpecFactoryForBestEffortRollup( - boolean overwrite, - Interval interval - ) - { - if (overwrite && !isChangeSegmentGranularity()) { - final OverwritingRootGenerationPartitions overwritingSegmentMeta = Preconditions.checkNotNull( - getOverwritingSegmentMeta(interval), - "Can't find overwritingSegmentMeta for interval[%s]", - interval - ); - return Pair.of( - new NumberedOverwritingShardSpecFactory( - overwritingSegmentMeta.getStartRootPartitionId(), - overwritingSegmentMeta.getEndRootPartitionId(), - overwritingSegmentMeta.getMinorVersionForNewSegments() - ), - null - ); - } else { - return Pair.of(NumberedShardSpecFactory.instance(), null); - } - } - private Map> collectIntervalsAndShardSpecs( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java index 2e20d59b5a4c..d2aced9de75e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java @@ -29,12 +29,10 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionPreconditions; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; @@ -70,36 +68,6 @@ public String getType() return "kill"; } - @Override - public boolean requireLockInputSegments() - { - return true; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - throws IOException - { - final List allSegments = new ArrayList<>(); - for (Interval interval : intervals) { - allSegments.addAll(taskActionClient.submit(new SegmentListUnusedAction(getDataSource(), interval))); - } - return allSegments; - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java index 12f117b94557..8f3cb08195f4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java @@ -28,15 +28,11 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -114,31 +110,6 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @JsonProperty("target") public Map getTargetLoadSpec() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java index 470b9956ac30..b400c3bd3bdb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NoopTask.java @@ -32,14 +32,8 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; 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.timeline.DataSegment; -import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; import java.util.Map; import java.util.UUID; @@ -160,31 +154,6 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception } } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public int getPriority() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 2d89589b1fb9..ce2b1c6cb65a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -44,7 +44,6 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.java.util.common.DateTimes; 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.CloseQuietly; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.FinalizeResultsQueryRunner; @@ -74,11 +73,8 @@ import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.util.Collections; -import java.util.List; import java.util.Map; import java.util.Timer; import java.util.TimerTask; @@ -490,31 +486,6 @@ public void run() return TaskStatus.success(getId()); } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); - } - @Override public boolean canRestore() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java index b5fad4d92b5c..a0269f6cc520 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java @@ -23,7 +23,7 @@ import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.task.AbstractTask.OverwritingRootGenerationPartitions; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask.OverwritingRootGenerationPartitions; import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java index 340b3cd050f9..8ee87a673721 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java @@ -27,16 +27,12 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -123,29 +119,4 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception return TaskStatus.success(getId()); } - - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index a7640c267256..8559ae04b5f2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -37,7 +37,7 @@ import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; @@ -88,7 +88,7 @@ * generates and pushes segments, and reports them to the {@link ParallelIndexSupervisorTask} instead of * publishing on its own. */ -public class ParallelIndexSubTask extends AbstractTask +public class ParallelIndexSubTask extends AbstractBatchIndexTask { public static final String TYPE = "index_sub"; @@ -231,6 +231,12 @@ public boolean changeSegmentGranularity(List intervalOfExistingSegment return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); } + @Override + public boolean isPerfectRollup() + { + return false; + } + @Nullable @Override public Granularity getSegmentGranularity(Interval interval) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 6285cea27cbf..e9746d8d3666 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -39,7 +39,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; -import org.apache.druid.indexing.common.task.AbstractTask; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; @@ -93,7 +93,7 @@ * * @see ParallelIndexTaskRunner */ -public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHandler +public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implements ChatHandler { public static final String TYPE = "index_parallel"; @@ -244,6 +244,12 @@ public boolean changeSegmentGranularity(List intervalOfExistingSegment return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); } + @Override + public boolean isPerfectRollup() + { + return false; + } + @Nullable @Override public Granularity getSegmentGranularity(Interval interval) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 6d170ba56b08..caba20941e79 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -64,13 +64,16 @@ import java.util.stream.StreamSupport; /** - * Remembers which activeTasks have locked which intervals. Tasks are permitted to lock an interval if no other task - * outside their group has locked an overlapping interval for the same datasource. When a task locks an interval, - * it is assigned a version string that it can use to publish segments. + * Remembers which activeTasks have locked which intervals or which segments. Tasks are permitted to lock an interval + * or a segment if no other task outside their group has locked an overlapping interval for the same datasource or + * the same segments. Note that TaskLockbox is also responsible for allocating segmentIds when a task requests to lock + * a new segment. Task lock might involve version assignment. * - * // TODO: how to use this? - * // with segemnt lock - * // note it also does segment allocation for segment lock + * - When a task locks an interval or a new segment, it is assigned a new version string that it can use to publish + * segments. + * - When a task locks a existing segment, it doesn't need to be assigned a new version. + * + * Note that tasks of higher priorities can revoke locks of tasks of lower priorities. */ public class TaskLockbox { @@ -428,15 +431,19 @@ private TaskLockPosse createOrFindLockPosse(LockRequest request) // Any number of shared locks can be acquired for the same dataSource and interval. return createNewTaskLockPosse(request); } else { + // During a rolling update, tasks of mixed versions can be run at the same time. Old tasks would request + // timeChunkLocks while new tasks would ask segmentLocks. The below check is to allow for old and new tasks + // to get locks of different granularities if they have the same groupId. final boolean allDifferentGranularity = conflictPosses .stream() .allMatch( conflictPosse -> conflictPosse.taskLock.getGranularity() != request.getGranularity() && conflictPosse.getTaskLock().getGroupId().equals(request.getGroupId()) - && conflictPosse.getTaskLock().getInterval().equals(request.getInterval()) // TODO: contains? + && conflictPosse.getTaskLock().getInterval().equals(request.getInterval()) ); if (allDifferentGranularity) { - // We can add a new taskLockPosse + // Lock collision was because of the different granularity in the same group. + // We can add a new taskLockPosse. return createNewTaskLockPosse(request); } else { if (isAllRevocable(conflictPosses, request.getPriority())) { @@ -532,6 +539,11 @@ public T doInCriticalSection(Task task, List intervals, CriticalAc } } + /** + * Check all locks task acquired are still valid. + * It doesn't check other semantics like acquired locks are enough to overwrite existing segments. + * This kind of semantic should be checked in each caller of {@link #doInCriticalSection}. + */ private boolean isTaskLocksValid(Task task, List intervals) { giant.lock(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 69a17045b78d..007879cd37e3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -41,7 +41,6 @@ import org.apache.druid.indexing.seekablestream.utils.RandomIdUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.NoopQueryRunner; @@ -55,15 +54,11 @@ import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; import org.apache.druid.utils.CircularBuffer; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.List; import java.util.Map; @@ -298,29 +293,4 @@ public SeekableStreamIndexTaskRunner getRun { return runner; } - - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegment) - { - return false; - } - - @Override - @Nullable - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java index 8305433e86ac..7879d8753569 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestTasks.java @@ -27,13 +27,6 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.timeline.DataSegment; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; public class TestTasks { @@ -75,31 +68,6 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) { @@ -128,31 +96,6 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 8fb3eefeec75..d01ef869532c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -54,7 +54,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NumberedOverwritingShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.joda.time.Interval; @@ -169,7 +169,7 @@ public void testRun() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedOverwritingShardSpec(32768, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); } } @@ -251,7 +251,7 @@ public void testRunCompactionTwiceWithKeepSegmentGranularity() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); Assert.assertEquals( - new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), + new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec() ); } @@ -271,7 +271,7 @@ public void testRunCompactionTwiceWithKeepSegmentGranularity() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); Assert.assertEquals( - new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, 0, 2, (short) 2, (short) 1), + new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, 0, 2, (short) 2, (short) 1), segments.get(i).getShardSpec() ); } @@ -359,7 +359,7 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); Assert.assertEquals( - new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), + new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec() ); } @@ -501,7 +501,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception segments.get(i).getInterval() ); Assert.assertEquals( - new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), + new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), segments.get(i).getShardSpec() ); } @@ -562,7 +562,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio segments.get(i).getInterval() ); Assert.assertEquals( - new NumberedOverwritingShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), + new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), segments.get(i).getShardSpec() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java index a46a5d6268df..54c07876bec2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java @@ -87,6 +87,12 @@ public boolean changeSegmentGranularity(List intervalOfExistingSegment return true; } + @Override + public boolean isPerfectRollup() + { + return true; + } + @Nullable @Override public Granularity getSegmentGranularity(Interval interval) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java index 6a23f43134b1..8724ba9c125a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java @@ -33,13 +33,10 @@ import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.Assert; -import javax.annotation.Nullable; -import java.util.Collections; import java.util.List; /** @@ -63,31 +60,6 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 779baf9c2678..8df0f36e3ff4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -32,7 +32,6 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.segment.loading.NoopDataSegmentArchiver; import org.apache.druid.segment.loading.NoopDataSegmentKiller; @@ -42,9 +41,7 @@ import org.apache.druid.server.coordination.NoopDataSegmentAnnouncer; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; -import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -52,11 +49,9 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Collections; -import java.util.List; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -186,31 +181,6 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 374b338accc1..a0a2e47e1854 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -53,7 +53,6 @@ import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.SegmentInsertAction; -import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; @@ -83,7 +82,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -144,7 +142,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -936,31 +933,6 @@ public String getType() return "test"; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { @@ -1004,32 +976,6 @@ public String getType() return "test"; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { @@ -1064,31 +1010,6 @@ public String getType() return "test"; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 4ef6c1d40a57..75a741504fe7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -44,7 +44,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; @@ -54,7 +53,6 @@ import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; @@ -70,7 +68,6 @@ import org.junit.Test; import org.junit.rules.ExpectedException; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -1236,31 +1233,6 @@ public boolean isReady(TaskActionClient taskActionClient) return true; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 20d77c732c7a..89824b046f35 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -41,7 +41,6 @@ import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; @@ -51,7 +50,6 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.server.security.Resource; -import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -65,7 +63,6 @@ import org.junit.Test; import org.junit.rules.ExpectedException; -import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; @@ -1025,31 +1022,6 @@ public boolean isReady(TaskActionClient taskActionClient) return false; } - @Override - public boolean requireLockInputSegments() - { - return false; - } - - @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) - { - return Collections.emptyList(); - } - - @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) - { - return false; - } - - @Nullable - @Override - public Granularity getSegmentGranularity(Interval interval) - { - return null; - } - @Override public TaskStatus run(TaskToolbox toolbox) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java index 0ab3ed2d35e4..0673571ae380 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -22,7 +22,7 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.OverwritingShardSpec; +import org.apache.druid.timeline.partition.OverwriteShardSpec; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -69,10 +69,10 @@ static Set annotateAtomicUpdateGroupSize(Set segments) for (Entry> entry : intervalToSegments.entrySet()) { final Interval interval = entry.getKey(); final List segmentsPerInterval = entry.getValue(); - final boolean isNonFirstGeneration = segmentsPerInterval.get(0).getShardSpec() instanceof OverwritingShardSpec; + final boolean isNonFirstGeneration = segmentsPerInterval.get(0).getShardSpec() instanceof OverwriteShardSpec; final boolean anyMismatch = segmentsPerInterval.stream().anyMatch( - segment -> (segment.getShardSpec() instanceof OverwritingShardSpec) != isNonFirstGeneration + segment -> (segment.getShardSpec() instanceof OverwriteShardSpec) != isNonFirstGeneration ); if (anyMismatch) { throw new ISE( @@ -80,10 +80,10 @@ static Set annotateAtomicUpdateGroupSize(Set segments) + "segments with non-overwritingShardSpec: [%s]," + "segments with overwritingShardSpec: [%s]", segmentsPerInterval.stream() - .filter(segment -> !(segment.getShardSpec() instanceof OverwritingShardSpec)) + .filter(segment -> !(segment.getShardSpec() instanceof OverwriteShardSpec)) .collect(Collectors.toList()), segmentsPerInterval.stream() - .filter(segment -> segment.getShardSpec() instanceof OverwritingShardSpec) + .filter(segment -> segment.getShardSpec() instanceof OverwriteShardSpec) .collect(Collectors.toList()) ); } @@ -96,7 +96,7 @@ static Set annotateAtomicUpdateGroupSize(Set segments) segmentsPerInterval .stream() .map(segment -> { - final OverwritingShardSpec shardSpec = (OverwritingShardSpec) segment.getShardSpec(); + final OverwriteShardSpec shardSpec = (OverwriteShardSpec) segment.getShardSpec(); return segment.withShardSpec(shardSpec.withAtomicUpdateGroupSize((short) segmentsPerInterval.size())); }) .collect(Collectors.toList()) diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index c2bc840dac94..ea4e69efbe60 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -37,7 +37,7 @@ import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NoneShardSpec; -import org.apache.druid.timeline.partition.NumberedOverwritingShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; @@ -992,7 +992,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce null, Collections.emptyList(), Collections.emptyList(), - ((NumberedOverwritingShardSpec) identifier.getShardSpec()).withAtomicUpdateGroupSize(1), + ((NumberedOverwriteShardSpec) identifier.getShardSpec()).withAtomicUpdateGroupSize(1), 0, 10L ) @@ -1021,7 +1021,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce null, Collections.emptyList(), Collections.emptyList(), - new NumberedOverwritingShardSpec( + new NumberedOverwriteShardSpec( 9 + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 1, diff --git a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java index 172c6d705954..90110695de93 100644 --- a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java @@ -29,7 +29,7 @@ import org.apache.druid.timeline.Overshadowable; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.NumberedOverwritingShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; @@ -215,7 +215,7 @@ public void testCompatible() { final NumberedShardSpec spec = new NumberedShardSpec(0, 0); Assert.assertTrue(spec.isCompatible(NumberedShardSpec.class)); - Assert.assertTrue(spec.isCompatible(NumberedOverwritingShardSpec.class)); + Assert.assertTrue(spec.isCompatible(NumberedOverwriteShardSpec.class)); } private static final class OvershadowableString implements Overshadowable From 7140f0c26527c7b1a85a474bf6d028823b0cc66f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 26 Apr 2019 16:57:56 -0700 Subject: [PATCH 15/52] fix build --- .../apache/druid/metadata/SQLMetadataSegmentManager.java | 8 ++++---- .../druid/metadata/SQLMetadataSegmentManagerTest.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index 35843c95b9dd..72b363818d48 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -397,10 +397,10 @@ private int enableSegments( Batch batch = handle.createBatch(); segments .stream() - .map(segment -> segment.getId()) - .filter(segmentId -> !versionedIntervalTimeline.isOvershadowed( - segmentId.getInterval(), - segmentId.getVersion() + .filter(segment -> !versionedIntervalTimeline.isOvershadowed( + segment.getInterval(), + segment.getMajorVersion(), + segment )) .forEach(segmentId -> batch.add( StringUtils.format( diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java index e7964481688a..e65e752928a9 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java @@ -99,7 +99,7 @@ private void publish(DataSegment segment, boolean used) throws IOException segment.getInterval().getStart().toString(), segment.getInterval().getEnd().toString(), (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, - segment.getVersion(), + segment.getMajorVersion(), used, jsonMapper.writeValueAsBytes(segment) ); From 7a471125eac8d1c4838b7ff4a989268a61bcdd53 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 30 Apr 2019 12:09:10 -0700 Subject: [PATCH 16/52] fix SQLMetadataSegmentManager --- .../org/apache/druid/metadata/SQLMetadataSegmentManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index 72b363818d48..8a8243eaa3f8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -402,11 +402,11 @@ private int enableSegments( segment.getMajorVersion(), segment )) - .forEach(segmentId -> batch.add( + .forEach(segment -> batch.add( StringUtils.format( "UPDATE %s SET used=true WHERE id = '%s'", getSegmentsTable(), - segmentId + segment.getId() ) )); return batch.execute().length; From 2305a466ad4db60dc192d0849e8bd8b74e21df5b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 1 May 2019 16:42:43 -0700 Subject: [PATCH 17/52] fix findInputSegments --- .../common/actions/SegmentListUsedAction.java | 3 +- .../common/task/AbstractBatchIndexTask.java | 53 +++++++++--------- .../indexing/common/task/CompactionTask.java | 8 +-- .../druid/indexing/common/task/IndexTask.java | 55 +++++++++++++++++-- .../batch/parallel/ParallelIndexSubTask.java | 12 ++-- .../parallel/ParallelIndexSupervisorTask.java | 10 +++- .../IngestSegmentFirehoseFactory.java | 2 + 7 files changed, 96 insertions(+), 47 deletions(-) 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 50c22d33d2af..461db0ae624f 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.timeline.DataSegment; import org.joda.time.Interval; +import java.util.Collection; import java.util.List; public class SegmentListUsedAction implements TaskAction> @@ -44,7 +45,7 @@ public class SegmentListUsedAction implements TaskAction> public SegmentListUsedAction( @JsonProperty("dataSource") String dataSource, @Deprecated @JsonProperty("interval") Interval interval, - @JsonProperty("intervals") List intervals + @JsonProperty("intervals") Collection intervals ) { this.dataSource = dataSource; 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 index 7d6ec9d8730d..2f7914a38f4b 100644 --- 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 @@ -124,13 +124,7 @@ public abstract List findInputSegments(TaskActionClient taskActionC @Nullable public abstract Granularity getSegmentGranularity(Interval interval); - boolean tryLockWithIntervals(TaskActionClient client, Set intervals) - throws IOException - { - return tryLockWithIntervals(client, new ArrayList<>(intervals)); - } - - protected boolean tryLockWithIntervals(TaskActionClient client, List intervals) + protected boolean tryLockWithIntervals(TaskActionClient client, List intervals, boolean isInitialRequest) throws IOException { if (requireLockInputSegments()) { @@ -141,7 +135,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i // However, tasks wouldn't know about new segments added after findInputSegments() call, it may missing those // segments. This is usually fine, but if you want to avoid this, you should use timeChunk lock instead. if (!intervals.isEmpty()) { - return tryLockWithSegments(client, findInputSegments(client, intervals)); + return tryLockWithSegments(client, findInputSegments(client, intervals), isInitialRequest); } else { return true; } @@ -152,7 +146,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } - boolean tryLockWithSegments(TaskActionClient client, List segments) throws IOException + boolean tryLockWithSegments(TaskActionClient client, List segments, boolean isInitialRequest) throws IOException { if (segments.isEmpty()) { changeSegmentGranularity = false; @@ -204,7 +198,9 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) for (DataSegment segment : segmentsToLock) { intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); } - intervalToSegments.values().forEach(this::verifyAndFindRootPartitionRangeAndMinorVersion); + intervalToSegments.values().forEach( + segmentsToCheck -> verifyAndFindRootPartitionRangeAndMinorVersion(segmentsToCheck, isInitialRequest) + ); final Closer lockCloserOnError = Closer.create(); for (Entry> entry : intervalToSegments.entrySet()) { final Interval interval = entry.getKey(); @@ -227,7 +223,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) ))); - if (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk())) { + if (isInitialRequest && (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk()))) { lockCloserOnError.close(); return false; } @@ -250,13 +246,23 @@ boolean tryLockWithSegments(TaskActionClient client, List segments) * - Are rootPartition range of inputSegments adjacent? Two rootPartition ranges are adjacent if they are consecutive. * - All atomicUpdateGroups of inputSegments must be full. (See {@code AtomicUpdateGroup#isFull()}). */ - private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments) + private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments, boolean isInitialRequest) { if (inputSegments.isEmpty()) { return; } - final List sortedSegments = verifyInputSegmentsToOverwritePerInterval(inputSegments); + final List sortedSegments = new ArrayList<>(inputSegments); + sortedSegments.sort((s1, s2) -> { + if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) { + return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId()); + } else { + return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId()); + } + }); + if (isInitialRequest) { + verifyInputSegmentsToOverwritePerInterval(sortedSegments); + } final Interval interval = sortedSegments.get(0).getInterval(); final short prevMaxMinorVersion = (short) sortedSegments .stream() @@ -301,25 +307,16 @@ public boolean isOverwriteMode() return !overwritingRootGenPartitions.isEmpty(); } - private static List verifyInputSegmentsToOverwritePerInterval(List inputSegments) + private static void verifyInputSegmentsToOverwritePerInterval(List sortedSegments) { - if (inputSegments.isEmpty()) { - return inputSegments; + if (sortedSegments.isEmpty()) { + return; } Preconditions.checkArgument( - inputSegments.stream().allMatch(segment -> segment.getInterval().equals(inputSegments.get(0).getInterval())) + sortedSegments.stream().allMatch(segment -> segment.getInterval().equals(sortedSegments.get(0).getInterval())) ); - final List sortedSegments = new ArrayList<>(inputSegments); - sortedSegments.sort((s1, s2) -> { - if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) { - return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId()); - } else { - return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId()); - } - }); - short atomicUpdateGroupSize = 1; // sanity check for (int i = 0; i < sortedSegments.size() - 1; i++) { @@ -347,10 +344,10 @@ private static List verifyInputSegmentsToOverwritePerInterval(List< atomicUpdateGroupSize = 1; } } - if (atomicUpdateGroupSize == sortedSegments.get(sortedSegments.size() - 1).getAtomicUpdateGroupSize()) { + if (atomicUpdateGroupSize != sortedSegments.get(sortedSegments.size() - 1).getAtomicUpdateGroupSize()) { throw new ISE("All atomicUpdateGroup must be compacted together"); } - return sortedSegments; + return; } } 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 1e256d69bfe6..a009280f65e7 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 @@ -271,17 +271,11 @@ public int getPriority() return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_MERGE_TASK_PRIORITY); } - @VisibleForTesting - SegmentProvider getSegmentProvider() - { - return segmentProvider; - } - @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { final List segments = segmentProvider.checkAndGetSegments(taskActionClient); - return tryLockWithSegments(taskActionClient, segments); + return tryLockWithSegments(taskActionClient, segments, true); } @Override 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 c5491e9fd767..8b8b8330e040 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 @@ -51,6 +51,8 @@ import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; +import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; +import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; @@ -118,6 +120,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; public class IndexTask extends AbstractBatchIndexTask implements ChatHandler { @@ -268,7 +271,51 @@ public boolean requireLockInputSegments() public List findInputSegments(TaskActionClient taskActionClient, List intervals) throws IOException { - return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); + return findInputSegments( + getDataSource(), + taskActionClient, + intervals, + ingestionSchema.ioConfig.firehoseFactory + ); + } + + public static List findInputSegments( + String dataSource, + TaskActionClient actionClient, + List intervalsToFind, + FirehoseFactory firehoseFactory + ) throws IOException + { + if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { + final List 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 inputSegmentIds = inputSegments.stream() + .map(WindowedSegmentId::getSegmentId) + .collect(Collectors.toList()); + final List 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, intervalsToFind)); + } } @Override @@ -300,7 +347,7 @@ private boolean tryLockIfNecessary(TaskActionClient actionClient, Collection(intervals)); + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals), true); } @GET @@ -463,7 +510,7 @@ public TaskStatus run(final TaskToolbox toolbox) maxRowsPerSegment ); - final Set allocateIntervals = allocateSpec.keySet(); + final List allocateIntervals = new ArrayList<>(allocateSpec.keySet()); // get locks for found shardSpec intervals if (!tryLockIfNecessary(toolbox.getTaskActionClient(), allocateIntervals)) { throw new ISE("Failed to get a lock for segments"); @@ -471,7 +518,7 @@ public TaskStatus run(final TaskToolbox toolbox) final DataSchema dataSchema; if (determineIntervals) { - if (!tryLockWithIntervals(toolbox.getTaskActionClient(), allocateIntervals)) { + if (!tryLockWithIntervals(toolbox.getTaskActionClient(), allocateIntervals, true)) { throw new ISE("Failed to get locks for intervals[%s]", allocateIntervals); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 8559ae04b5f2..33ef9a0c6328 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -34,7 +34,6 @@ import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; @@ -158,7 +157,7 @@ public boolean isReady(TaskActionClient taskActionClient) private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) { try { - return tryLockWithIntervals(actionClient, new ArrayList<>(intervals)); + return tryLockWithIntervals(actionClient, new ArrayList<>(intervals), false); } catch (Exception e) { log.error(e, "Failed to acquire locks for intervals[%s]", intervals); @@ -221,7 +220,12 @@ public boolean requireLockInputSegments() public List findInputSegments(TaskActionClient taskActionClient, List intervals) throws IOException { - return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); + return IndexTask.findInputSegments( + getDataSource(), + taskActionClient, + intervals, + ingestionSchema.getIOConfig().getFirehoseFactory() + ); } @Override @@ -421,7 +425,7 @@ private List generateAndPushSegments( } else { final Granularity segmentGranularity = findSegmentGranularity(granularitySpec); final Interval timeChunk = segmentGranularity.bucket(inputRow.getTimestamp()); - if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk))) { + if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk), false)) { throw new ISE("Failed to get locks for interval[%s]", timeChunk); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index e9746d8d3666..6910ca96d75c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -35,7 +35,6 @@ import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -221,14 +220,19 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception .getGranularitySpec() .bucketIntervals(); - return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get())); + return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get()), true); } @Override public List findInputSegments(TaskActionClient taskActionClient, List intervals) throws IOException { - return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); + return IndexTask.findInputSegments( + getDataSource(), + taskActionClient, + intervals, + ingestionSchema.getIOConfig().getFirehoseFactory() + ); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 1e24a6687364..1c67a4008b8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -159,12 +159,14 @@ public String getDataSource() } @JsonProperty + @Nullable public Interval getInterval() { return interval; } @JsonProperty + @Nullable public List getSegments() { return segmentIds; From 28890b7e0485503b3b672544778fb20ccbd2b3b0 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 1 May 2019 17:54:54 -0700 Subject: [PATCH 18/52] adding more tests --- .../parallel/ParallelIndexSupervisorTask.java | 16 +-- .../indexing/common/task/IndexTaskTest.java | 133 ++++++++++++++++-- ...stractParallelIndexSupervisorTaskTest.java | 8 ++ ...rallelIndexSupervisorTaskResourceTest.java | 1 - .../ParallelIndexSupervisorTaskTest.java | 93 ++++++------ .../indexing/overlord/TaskLockboxTest.java | 2 - ...exerSQLMetadataStorageCoordinatorTest.java | 1 - 7 files changed, 185 insertions(+), 69 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 6910ca96d75c..03bff95c79c6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -410,10 +410,6 @@ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException { final String dataSource = getDataSource(); final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec(); - final SortedSet bucketIntervals = Preconditions.checkNotNull( - granularitySpec.bucketIntervals().orNull(), - "bucketIntervals" - ); // List locks whenever allocating a new segment because locks might be revoked and no longer valid. final Map versions = toolbox .getTaskActionClient() @@ -421,15 +417,9 @@ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException .stream() .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); - if (!maybeInterval.isPresent()) { - throw new IAE("Could not find interval for timestamp [%s]", timestamp); - } - - final Interval interval = maybeInterval.get(); - if (!bucketIntervals.contains(interval)) { - throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); - } + final Interval interval = granularitySpec + .bucketInterval(timestamp) + .or(granularitySpec.getSegmentGranularity().bucket(timestamp)); final int partitionNum = Counters.getAndIncrementInt(partitionNumCountersPerInterval, interval); return new SegmentIdWithShardSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 0a89b49ba3e1..f7c9cef75167 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -48,6 +48,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -57,7 +58,6 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; @@ -76,7 +76,9 @@ import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -98,7 +100,6 @@ import java.util.Map; import java.util.Set; -// TODO: change segmentGranularity public class IndexTaskTest extends IngestionTestBase { @Rule @@ -128,16 +129,13 @@ public class IndexTaskTest extends IngestionTestBase private static final IndexSpec indexSpec = new IndexSpec(); private final ObjectMapper jsonMapper; - private IndexMergerV9 indexMergerV9; - private IndexIO indexIO; - private RowIngestionMetersFactory rowIngestionMetersFactory; + private final IndexIO indexIO; + private final RowIngestionMetersFactory rowIngestionMetersFactory; private TestTaskRunner taskRunner; public IndexTaskTest() { jsonMapper = getObjectMapper(); - - indexMergerV9 = getIndexMerger(); indexIO = getIndexIO(); rowIngestionMetersFactory = getRowIngestionMetersFactory(); } @@ -941,11 +939,15 @@ public void testMultipleParseExceptionsSuccess() throws Exception try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { writer.write("{\"time\":\"unparseable\",\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // unparseable time writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // valid row - writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":\"notnumber\",\"dimFloat\":3.0,\"val\":1}\n"); // row with invalid long dimension - writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":\"notnumber\",\"val\":1}\n"); // row with invalid float dimension - writer.write("{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":4.0,\"val\":\"notnumber\"}\n"); // row with invalid metric + writer.write( + "{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":\"notnumber\",\"dimFloat\":3.0,\"val\":1}\n"); // row with invalid long dimension + writer.write( + "{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":\"notnumber\",\"val\":1}\n"); // row with invalid float dimension + writer.write( + "{\"time\":\"2014-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":4.0,\"val\":\"notnumber\"}\n"); // row with invalid metric writer.write("{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // invalid JSON - writer.write("{\"time\":\"3014-03-01T00:00:10Z\",\"dim\":\"outsideofinterval\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // thrown away + writer.write( + "{\"time\":\"3014-03-01T00:00:10Z\",\"dim\":\"outsideofinterval\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // thrown away writer.write("{\"time\":\"99999999999-01-01T00:00:10Z\",\"dim\":\"b\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}\n"); // unparseable time writer.write("this is not JSON\n"); // invalid JSON } @@ -1437,6 +1439,115 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception Assert.assertEquals(expectedUnparseables, reportData.getUnparseableEvents()); } + @Test + public void testOverwriteWithSameSegmentGranularity() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + populateRollupTestData(tmpFile); + + for (int i = 0; i < 2; i++) { + final IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + jsonMapper, + tmpDir, + null, + new UniformGranularitySpec( + Granularities.DAY, + Granularities.DAY, + true, + null + ), + createTuningConfig(3, 2, null, 2L, null, null, false, true), + false + ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null, + rowIngestionMetersFactory + ); + + final List segments = runTask(indexTask).rhs; + + Assert.assertEquals(5, segments.size()); + + final Interval expectedInterval = Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"); + for (int j = 0; j < 5; j++) { + final DataSegment segment = segments.get(j); + Assert.assertEquals("test", segment.getDataSource()); + Assert.assertEquals(expectedInterval, segment.getInterval()); + if (i == 0) { + Assert.assertEquals(NumberedShardSpec.class, segment.getShardSpec().getClass()); + Assert.assertEquals(j, segment.getShardSpec().getPartitionNum()); + } else { + Assert.assertEquals(NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass()); + final NumberedOverwriteShardSpec numberedOverwriteShardSpec = + (NumberedOverwriteShardSpec) segment.getShardSpec(); + Assert.assertEquals( + j + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, + numberedOverwriteShardSpec.getPartitionNum() + ); + Assert.assertEquals(1, numberedOverwriteShardSpec.getMinorVersion()); + Assert.assertEquals(5, numberedOverwriteShardSpec.getAtomicUpdateGroupSize()); + Assert.assertEquals(0, numberedOverwriteShardSpec.getStartRootPartitionId()); + Assert.assertEquals(5, numberedOverwriteShardSpec.getEndRootPartitionId()); + } + } + } + } + + @Test + public void testOverwriteWithDifferentSegmentGranularity() throws Exception + { + final File tmpDir = temporaryFolder.newFolder(); + final File tmpFile = File.createTempFile("druid", "index", tmpDir); + + populateRollupTestData(tmpFile); + + for (int i = 0; i < 2; i++) { + final Granularity segmentGranularity = i == 0 ? Granularities.DAY : Granularities.MONTH; + final IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + jsonMapper, + tmpDir, + null, + new UniformGranularitySpec( + segmentGranularity, + Granularities.DAY, + true, + null + ), + createTuningConfig(3, 2, null, 2L, null, null, false, true), + false + ), + null, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, + null, + rowIngestionMetersFactory + ); + + final List segments = runTask(indexTask).rhs; + + Assert.assertEquals(5, segments.size()); + + final Interval expectedInterval = i == 0 + ? Intervals.of("2014-01-01/2014-01-02") + : Intervals.of("2014-01-01/2014-02-01"); + for (int j = 0; j < 5; j++) { + final DataSegment segment = segments.get(j); + Assert.assertEquals("test", segment.getDataSource()); + Assert.assertEquals(expectedInterval, segment.getInterval()); + Assert.assertEquals(NumberedShardSpec.class, segment.getShardSpec().getClass()); + Assert.assertEquals(j, segment.getShardSpec().getPartitionNum()); + } + } + } + public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus status) { // full stacktrace will be too long and make tests brittle (e.g. if line # changes), just match the main message diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index e1ba95871d23..30c932c06629 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -52,11 +52,13 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentKiller; +import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.security.AllowAllAuthorizer; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Rule; import org.junit.rules.TemporaryFolder; @@ -355,6 +357,12 @@ static class LocalParallelIndexTaskClient extends ParallelIndexTaskClient this.supervisorTask = supervisorTask; } + @Override + public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException + { + return supervisorTask.allocateNewSegment(timestamp); + } + @Override public void report(String supervisorTaskId, List pushedSegments) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index db7418774a13..a04bdc4802c7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -676,7 +676,6 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception 0, 1L ); - // TODO: maybe check shardSpec?? taskClient.report(getId(), Collections.singletonList(segment)); return TaskStatus.fromCode(getId(), state); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index e82f479a7599..cde4679faa3b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -30,12 +30,15 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.PartitionChunk; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -55,8 +58,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.stream.Collectors; -// TODO: change segmentGranularity public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest { private File inputDir; @@ -130,10 +133,11 @@ public void testIsReady() throws Exception } } - private void runTestWithoutIntervalTask() throws Exception + private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity) throws Exception { final ParallelIndexSupervisorTask task = newTask( - null, + interval, + segmentGranularity, new ParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false @@ -148,58 +152,54 @@ private void runTestWithoutIntervalTask() throws Exception shutdownTask(task); } - @Test - public void testWithoutInterval() throws Exception + private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity secondSegmentGranularity) throws Exception { // Ingest all data. - runTestWithoutIntervalTask(); + runTestTask(inputInterval, Granularities.DAY); // Read the segments for one day. final Interval interval = Intervals.of("2017-12-24/P5D"); - final List allOldSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); - final Map> oldIntervalToSegments = new HashMap<>(); - allOldSegments.forEach( - segment -> oldIntervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) - ); + final List allSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); // Reingest the same data. Each segment should get replaced by a segment with a newer version. - runTestWithoutIntervalTask(); + runTestTask(inputInterval, secondSegmentGranularity); // Verify that the segment has been replaced. - final List allNewSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); - final Map> newIntervalToSegments = new HashMap<>(); - allNewSegments.forEach( - segment -> newIntervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) - ); - Assert.assertEquals(allOldSegments.size(), allNewSegments.size()); - for (Entry> entry : oldIntervalToSegments.entrySet()) { - final List oldSegments = entry.getValue(); - final List newSegments = newIntervalToSegments.get(entry.getKey()); - Assert.assertEquals(oldSegments.size(), newSegments.size()); - newSegments.forEach( - newSegment -> Assert.assertTrue( - oldSegments.stream().allMatch(oldSegment -> oldSegment.getMinorVersion() < newSegment.getMinorVersion()) - ) - ); - } + final List newSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + allSegments.addAll(newSegments); + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(allSegments); + final List visibles = timeline.lookup(interval) + .stream() + .flatMap(holder -> holder.getObject().stream()) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); + Assert.assertEquals(newSegments, visibles); + } + + @Test + public void testWithoutInterval() throws Exception + { + testRunAndOverwrite(null, Granularities.DAY); } @Test() public void testRunInParallel() throws Exception { - final ParallelIndexSupervisorTask task = newTask( - Intervals.of("2017/2018"), - new ParallelIndexIOConfig( - new LocalFirehoseFactory(inputDir, "test_*", null), - false - ) - ); - actionClient = createActionClient(task); - toolbox = createTaskToolbox(task); + // Ingest all data. + testRunAndOverwrite(Intervals.of("2017/2018"), Granularities.DAY); + } - prepareTaskForLocking(task); - Assert.assertTrue(task.isReady(actionClient)); - Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); + @Test + public void testWithoutIntervalWithDifferentSegmentGranularity() throws Exception + { + testRunAndOverwrite(null, Granularities.MONTH); + } + + @Test() + public void testRunInParallelWithDifferentSegmentGranularity() throws Exception + { + // Ingest all data. + testRunAndOverwrite(Intervals.of("2017/2018"), Granularities.MONTH); } @Test @@ -250,6 +250,7 @@ public void testWith1MaxNumSubTasks() throws Exception { final ParallelIndexSupervisorTask task = newTask( Intervals.of("2017/2018"), + Granularities.DAY, new ParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), false @@ -286,13 +287,20 @@ public void testWith1MaxNumSubTasks() throws Exception Assert.assertNull("Runner must be null if the task was in the sequential mode", task.getRunner()); } + private ParallelIndexSupervisorTask newTask(Interval interval, ParallelIndexIOConfig ioConfig) + { + return newTask(interval, Granularities.DAY, ioConfig); + } + private ParallelIndexSupervisorTask newTask( Interval interval, + Granularity segmentGranularity, ParallelIndexIOConfig ioConfig ) { return newTask( interval, + segmentGranularity, ioConfig, new ParallelIndexTuningConfig( null, @@ -321,6 +329,7 @@ private ParallelIndexSupervisorTask newTask( private ParallelIndexSupervisorTask newTask( Interval interval, + Granularity segmentGranularity, ParallelIndexIOConfig ioConfig, ParallelIndexTuningConfig tuningConfig ) @@ -341,7 +350,8 @@ private ParallelIndexSupervisorTask newTask( new LongSumAggregatorFactory("val", "val") }, new UniformGranularitySpec( - Granularities.DAY, +// Granularities.DAY, + segmentGranularity, Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval) ), @@ -387,6 +397,7 @@ private static class TestSupervisorTask extends TestParallelIndexSupervisorTask @Override ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) { + setToolbox(toolbox); setRunner( new TestRunner( toolbox, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 75a741504fe7..0111d32a2c62 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -981,7 +981,6 @@ private void allocateSegmentsAndAssert( ShardSpecFactory shardSpecFactory ) { - // TODO: test overshadowingSegments for (int i = 0; i < numSegmentsToAllocate; i++) { final LockRequestForNewSegment request = new LockRequestForNewSegment( TaskLockType.EXCLUSIVE, @@ -1014,7 +1013,6 @@ private void assertAllocatedSegments( Assert.assertEquals(lockRequest.getInterval(), segmentLock.getInterval()); Assert.assertEquals(lockRequest.getShardSpecFactory().getShardSpecClass(), segmentId.getShardSpec().getClass()); Assert.assertEquals(lockRequest.getPriority(), lockRequest.getPriority()); - // TODO: fix this to check overwriting shardSpec } @Test diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index ea4e69efbe60..76d637c6df6d 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -958,7 +958,6 @@ public void testDeletePendingSegment() throws InterruptedException Assert.assertEquals(10, numDeleted); } - // TODO: fix this test to use overwriting shardSpec @Test public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOException { From af4697f611431156ce965403d419605c25449047 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 2 May 2019 16:14:51 -0700 Subject: [PATCH 19/52] fixing task lock checks --- .../SegmentTransactionalInsertAction.java | 13 +- .../SegmentTransactionalOverwriteAction.java | 216 ++++++++++++++++++ .../common/task/AbstractBatchIndexTask.java | 12 +- .../druid/indexing/common/task/IndexTask.java | 52 +++-- .../appenderator/AppenderatorImpl.java | 2 +- 5 files changed, 270 insertions(+), 25 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java 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 ba44154b3948..2af0899a34b0 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 @@ -24,6 +24,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; 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.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; @@ -40,7 +41,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; /** @@ -141,15 +141,16 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throw new RuntimeException(e); } + // Non-overwriting tasks should release locks as early as possible, so that other tasks can lock same segments. final List locks = toolbox.getTaskLockbox().findLocksForTask(task); - if (locks.get(0).getGranularity() == LockGranularity.SEGMENT) { - for (Entry> entry : intervalToPartitionIds.entrySet()) { - final Interval interval = entry.getKey(); - final Set partitionIds = entry.getValue(); - partitionIds.forEach(partitionId -> toolbox.getTaskLockbox().unlock(task, interval, partitionId)); + for (TaskLock lock : locks) { + if (lock.getGranularity() == LockGranularity.SEGMENT) { + final SegmentLock segmentLock = (SegmentLock) lock; + toolbox.getTaskLockbox().unlock(task, segmentLock.getInterval(), segmentLock.getPartitionId()); } } + // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java new file mode 100644 index 000000000000..098aefe73b8b --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +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.LockGranularity; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.CriticalAction; +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 java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class SegmentTransactionalOverwriteAction implements TaskAction +{ + private final Set oldSegments; + private final Set newSegments; + + @JsonCreator + public SegmentTransactionalOverwriteAction( + @JsonProperty("oldSegments") Set oldSegments, + @JsonProperty("newSegments") Set newSegments + ) + { + this.oldSegments = oldSegments; + this.newSegments = newSegments; + } + + @JsonProperty + public Set getOldSegments() + { + return oldSegments; + } + + @JsonProperty + public Set getNewSegments() + { + return newSegments; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) + { + final Set allSegments = new HashSet<>(oldSegments); + allSegments.addAll(newSegments); + TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); + + final Map> intervalToPartitionIds = new HashMap<>(); + for (DataSegment segment : allSegments) { + intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new HashSet<>()) + .add(segment.getShardSpec().getPartitionNum()); + } + + final List 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, + new ArrayList<>(intervalToPartitionIds.keySet()), + CriticalAction.builder() + .onValidLocks( + () -> toolbox.getIndexerMetadataStorageCoordinator() + .announceHistoricalSegments(newSegments, null, null) + ) + .onInvalidLocks( + () -> SegmentPublishResult.fail( + "Invalid task locks. Maybe they are revoked by a higher priority task." + + " Please check the overlord log for details." + ) + ) + .build() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + // Emit metrics + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); + IndexTaskUtils.setTaskDimensions(metricBuilder, task); + + if (retVal.isSuccess()) { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); + } else { + toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); + } + + // getSegments() should return an empty set if announceHistoricalSegments() failed + for (DataSegment segment : retVal.getSegments()) { + metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); + toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); + } + + return retVal; + } + + private void checkWithSegmentLock() + { + final Map> oldSegmentsMap = groupSegmentsByIntervalAndSort(oldSegments); + final Map> newSegmentsMap = groupSegmentsByIntervalAndSort(newSegments); + + oldSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); + newSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); + + oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> { + final List 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> groupSegmentsByIntervalAndSort(Set segments) + { + final Map> 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() + { + return false; + } + + @Override + public String toString() + { + return "SegmentTransactionalOverwriteAction{" + + "oldSegments=" + oldSegments + + ", newSegments=" + newSegments + + '}'; + } +} 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 index 2f7914a38f4b..05c62efa26c9 100644 --- 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 @@ -56,6 +56,8 @@ public abstract class AbstractBatchIndexTask extends AbstractTask { @Nullable private Map overwritingRootGenPartitions; + @Nullable + private Set allInputSegments; @Nullable private Boolean changeSegmentGranularity; @@ -194,6 +196,12 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, .map(PartitionChunk::getObject) .collect(Collectors.toList()); + if (allInputSegments == null) { + allInputSegments = new HashSet<>(segmentsToLock); + } else { + + } + final Map> intervalToSegments = new HashMap<>(); for (DataSegment segment : segmentsToLock) { intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); @@ -261,7 +269,7 @@ private void verifyAndFindRootPartitionRangeAndMinorVersion(List in } }); if (isInitialRequest) { - verifyInputSegmentsToOverwritePerInterval(sortedSegments); + verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(sortedSegments); } final Interval interval = sortedSegments.get(0).getInterval(); final short prevMaxMinorVersion = (short) sortedSegments @@ -307,7 +315,7 @@ public boolean isOverwriteMode() return !overwritingRootGenPartitions.isEmpty(); } - private static void verifyInputSegmentsToOverwritePerInterval(List sortedSegments) + public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List sortedSegments) { if (sortedSegments.isEmpty()) { return; 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 8b8b8330e040..48911a2d323b 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 @@ -47,6 +47,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalOverwriteAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -499,8 +500,10 @@ public TaskStatus run(final TaskToolbox toolbox) // Initialize maxRowsPerSegment and maxTotalRows lazily final IndexTuningConfig tuningConfig = ingestionSchema.tuningConfig; - @Nullable final Integer maxRowsPerSegment = getValidMaxRowsPerSegment(tuningConfig); - @Nullable final Long maxTotalRows = getValidMaxTotalRows(tuningConfig); + @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> allocateSpec = determineShardSpecs( @@ -625,7 +628,7 @@ private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningCon * shardSpecs by itself. Intervals must be determined if they are not specified in {@link GranularitySpec}. * ShardSpecs must be determined if the perfect rollup must be guaranteed even though the number of shards is not * specified in {@link IndexTuningConfig}. - *

+ *

* If both intervals and shardSpecs don't have to be determined, this method simply returns {@link ShardSpecs} for the * given intervals. Here, if {@link IndexTuningConfig#numShards} is not specified, {@link NumberedShardSpec} is used. *

@@ -688,7 +691,10 @@ private Map> createShardSpecWithoutInp // Overwrite mode, guaranteed rollup: shardSpecs must be known in advance. final int numShards = tuningConfig.getNumShards() == null ? 1 : tuningConfig.getNumShards(); for (Interval interval : intervals) { - allocateSpec.put(interval, createShardSpecFactoryForGuaranteedRollup(numShards, tuningConfig.partitionDimensions)); + allocateSpec.put( + interval, + createShardSpecFactoryForGuaranteedRollup(numShards, tuningConfig.partitionDimensions) + ); } } else { for (Interval interval : intervals) { @@ -744,7 +750,10 @@ private Map> createShardSpecsFromInput if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) { // Overwrite mode, guaranteed rollup: # of shards must be known in advance. - allocateSpecs.put(interval, createShardSpecFactoryForGuaranteedRollup(numShards, tuningConfig.partitionDimensions)); + allocateSpecs.put( + interval, + createShardSpecFactoryForGuaranteedRollup(numShards, tuningConfig.partitionDimensions) + ); } else { allocateSpecs.put(interval, null); } @@ -903,7 +912,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( * If the number of rows added to {@link BaseAppenderatorDriver} so far exceeds {@link IndexTuningConfig#maxTotalRows} * * - * + *

* At the end of this method, all the remaining segments are published. * * @return true if generated segments are successfully published, otherwise false @@ -943,10 +952,17 @@ private TaskStatus generateAndPublishSegments( allocateSpec ); - final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); - return toolbox.getTaskActionClient().submit(action); - }; + final TransactionalSegmentPublisher publisher; + if (!isGuaranteedRollup && isOverwriteMode() && !isChangeSegmentGranularity()) { + publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient() + .submit(new SegmentTransactionalInsertAction(segments)); + } else { + + publisher = (segments, commitMetadata) -> { + + return toolbox.getTaskActionClient().submit(new SegmentTransactionalOverwriteAction()); + }; + } try ( final Appenderator appenderator = newAppenderator( @@ -1063,8 +1079,10 @@ private TaskStatus generateAndPublishSegments( */ public static Integer getValidMaxRowsPerSegment(IndexTuningConfig tuningConfig) { - @Nullable final Integer numShards = tuningConfig.numShards; - @Nullable final Integer maxRowsPerSegment = tuningConfig.maxRowsPerSegment; + @Nullable + final Integer numShards = tuningConfig.numShards; + @Nullable + final Integer maxRowsPerSegment = tuningConfig.maxRowsPerSegment; if (numShards == null || numShards == -1) { return maxRowsPerSegment == null || maxRowsPerSegment.equals(-1) ? IndexTuningConfig.DEFAULT_MAX_ROWS_PER_SEGMENT @@ -1081,8 +1099,10 @@ public static Integer getValidMaxRowsPerSegment(IndexTuningConfig tuningConfig) */ public static Long getValidMaxTotalRows(IndexTuningConfig tuningConfig) { - @Nullable final Integer numShards = tuningConfig.numShards; - @Nullable final Long maxTotalRows = tuningConfig.maxTotalRows; + @Nullable + final Integer numShards = tuningConfig.numShards; + @Nullable + final Long maxTotalRows = tuningConfig.maxTotalRows; if (numShards == null || numShards == -1) { return maxTotalRows == null ? IndexTuningConfig.DEFAULT_MAX_TOTAL_ROWS : maxTotalRows; } else { @@ -1401,8 +1421,8 @@ private IndexTuningConfig( ); this.maxRowsPerSegment = (maxRowsPerSegment != null && maxRowsPerSegment == -1) - ? null - : maxRowsPerSegment; + ? null + : maxRowsPerSegment; this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 1a431ca41c4e..7fc464a323a9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -272,7 +272,7 @@ public AppenderatorAddResult add( boolean isPersistRequired = false; boolean persist = false; - List persistReasons = new ArrayList(); + List persistReasons = new ArrayList<>(); if (!sink.canAppendRow()) { persist = true; From 948d335d86ee359fbb89c01a48da08006584748b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 3 May 2019 15:28:37 -0700 Subject: [PATCH 20/52] add SegmentTransactionalOverwriteAction --- .../SegmentTransactionalOverwriteAction.java | 2 + .../common/task/AbstractBatchIndexTask.java | 18 +++++---- .../druid/indexing/common/task/IndexTask.java | 14 ++++--- .../batch/parallel/ParallelIndexSubTask.java | 33 ++++++++++------ .../parallel/ParallelIndexSupervisorTask.java | 7 +--- .../parallel/ParallelIndexTaskClient.java | 6 +-- .../batch/parallel/PushedSegmentsReport.java | 21 +++++++--- .../SinglePhaseParallelIndexTaskRunner.java | 39 ++++++++++++------- .../common/task/IngestionTestBase.java | 3 ++ ...stractParallelIndexSupervisorTaskTest.java | 6 +-- ...rallelIndexSupervisorTaskResourceTest.java | 2 +- .../ParallelIndexSupervisorTaskTest.java | 3 -- .../TransactionalSegmentPublisher.java | 7 ++-- 13 files changed, 97 insertions(+), 64 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java index 098aefe73b8b..273c9acbd10c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; @@ -42,6 +43,7 @@ import java.util.Map; import java.util.Set; +// TODO: merging with SegmentTransactionalInsertAction? public class SegmentTransactionalOverwriteAction implements TaskAction { private final Set oldSegments; 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 index 05c62efa26c9..05dc23e34a68 100644 --- 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 @@ -143,6 +143,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } else { changeSegmentGranularity = false; + allInputSegments = Collections.emptySet(); overwritingRootGenPartitions = Collections.emptyMap(); return true; } @@ -152,6 +153,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, { if (segments.isEmpty()) { changeSegmentGranularity = false; + allInputSegments = Collections.emptySet(); overwritingRootGenPartitions = Collections.emptyMap(); return true; } @@ -162,6 +164,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, changeSegmentGranularity = changeSegmentGranularity(intervals); if (changeSegmentGranularity || isPerfectRollup()) { + allInputSegments = Collections.emptySet(); overwritingRootGenPartitions = Collections.emptyMap(); // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined final Set uniqueIntervals = new HashSet<>(); @@ -198,8 +201,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, if (allInputSegments == null) { allInputSegments = new HashSet<>(segmentsToLock); - } else { - + overwritingRootGenPartitions = new HashMap<>(); } final Map> intervalToSegments = new HashMap<>(); @@ -240,6 +242,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, } } else { changeSegmentGranularity = false; + allInputSegments = Collections.emptySet(); overwritingRootGenPartitions = Collections.emptyMap(); return true; } @@ -278,9 +281,6 @@ private void verifyAndFindRootPartitionRangeAndMinorVersion(List in .max() .orElseThrow(() -> new ISE("Empty inputSegments")); - if (overwritingRootGenPartitions == null) { - overwritingRootGenPartitions = new HashMap<>(); - } overwritingRootGenPartitions.put( interval, new OverwritingRootGenerationPartitions( @@ -291,11 +291,17 @@ private void verifyAndFindRootPartitionRangeAndMinorVersion(List in ); } + // TODO: perhaps merge the below methods protected boolean isChangeSegmentGranularity() { return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); } + public Set getAllInputSegments() + { + return Preconditions.checkNotNull(allInputSegments, "allInputSegments is not initialized"); + } + Map getAllOverwritingSegmentMeta() { Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); @@ -355,7 +361,5 @@ public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List< if (atomicUpdateGroupSize != sortedSegments.get(sortedSegments.size() - 1).getAtomicUpdateGroupSize()) { throw new ISE("All atomicUpdateGroup must be compacted together"); } - - return; } } 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 48911a2d323b..9e69f618e24d 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 @@ -955,13 +955,15 @@ private TaskStatus generateAndPublishSegments( final TransactionalSegmentPublisher publisher; if (!isGuaranteedRollup && isOverwriteMode() && !isChangeSegmentGranularity()) { publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient() - .submit(new SegmentTransactionalInsertAction(segments)); + .submit( + new SegmentTransactionalOverwriteAction( + getAllInputSegments(), + segments + ) + ); } else { - - publisher = (segments, commitMetadata) -> { - - return toolbox.getTaskActionClient().submit(new SegmentTransactionalOverwriteAction()); - }; + publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient() + .submit(new SegmentTransactionalInsertAction(segments)); } try ( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 33ef9a0c6328..24b71dc64e82 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -66,8 +66,10 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; +import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.Interval; @@ -76,11 +78,14 @@ 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.SortedSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; /** * A worker task of {@link ParallelIndexSupervisorTask}. Similar to {@link IndexTask}, but this task @@ -199,13 +204,23 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ingestionSchema.getTuningConfig().getChatHandlerTimeout(), ingestionSchema.getTuningConfig().getChatHandlerNumRetries() ); - final List pushedSegments = generateAndPushSegments( + final Set pushedSegments = generateAndPushSegments( toolbox, taskClient, firehoseFactory, firehoseTempDir ); - taskClient.report(supervisorTaskId, pushedSegments); + + // Find inputSegments overshadowed by pushedSegments + final Set allSegments = new HashSet<>(getAllInputSegments()); + allSegments.addAll(pushedSegments); + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(allSegments); + final Set oldSegments = timeline.findFullyOvershadowed() + .stream() + .flatMap(holder -> holder.getObject().stream()) + .map(PartitionChunk::getObject) + .collect(Collectors.toSet()); + taskClient.report(supervisorTaskId, oldSegments, pushedSegments); return TaskStatus.success(getId()); } @@ -249,10 +264,7 @@ public Granularity getSegmentGranularity(Interval interval) } @VisibleForTesting - SegmentAllocator createSegmentAllocator( - TaskToolbox toolbox, - ParallelIndexTaskClient taskClient - ) + SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexTaskClient taskClient) { return new WrappingSegmentAllocator(toolbox, taskClient); } @@ -269,10 +281,7 @@ private class WrappingSegmentAllocator implements SegmentAllocator */ private SegmentAllocator internalAllocator; - private WrappingSegmentAllocator( - TaskToolbox toolbox, - ParallelIndexTaskClient taskClient - ) + private WrappingSegmentAllocator(TaskToolbox toolbox, ParallelIndexTaskClient taskClient) { this.toolbox = toolbox; this.taskClient = taskClient; @@ -360,7 +369,7 @@ private SegmentAllocator createSegmentAllocator() * * @return true if generated segments are successfully published, otherwise false */ - private List generateAndPushSegments( + private Set generateAndPushSegments( final TaskToolbox toolbox, final ParallelIndexTaskClient taskClient, final FirehoseFactory firehoseFactory, @@ -397,7 +406,7 @@ private List generateAndPushSegments( ) { driver.startJob(null); - final List pushedSegments = new ArrayList<>(); + final Set pushedSegments = new HashSet<>(); while (firehose.hasMore()) { try { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 03bff95c79c6..9377e8b65f7d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -382,12 +382,7 @@ private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningC @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE) public Response allocateSegment(DateTime timestamp, @Context final HttpServletRequest req) { - ChatHandlers.authorizationCheck( - req, - Action.READ, - getDataSource(), - authorizerMapper - ); + ChatHandlers.authorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); if (toolbox == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java index 9a639d33bb57..3d2ed6b2d1e0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java @@ -33,7 +33,7 @@ import org.joda.time.Duration; import java.io.IOException; -import java.util.List; +import java.util.Set; public class ParallelIndexTaskClient extends IndexTaskClient { @@ -84,7 +84,7 @@ public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime } } - public void report(String supervisorTaskId, List pushedSegments) + public void report(String supervisorTaskId, Set oldSegments, Set pushedSegments) { try { final FullResponseHolder response = submitSmileRequest( @@ -92,7 +92,7 @@ public void report(String supervisorTaskId, List pushedSegments) HttpMethod.POST, "report", null, - serialize(new PushedSegmentsReport(subtaskId, pushedSegments)), + serialize(new PushedSegmentsReport(subtaskId, oldSegments, pushedSegments)), true ); if (!isSuccess(response)) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java index ddb1b48c386a..e83d9b025c09 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java @@ -24,7 +24,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.timeline.DataSegment; -import java.util.List; +import java.util.Set; /** * This class is used in native parallel batch indexing, currently only in {@link SinglePhaseParallelIndexTaskRunner}. @@ -34,16 +34,19 @@ public class PushedSegmentsReport { private final String taskId; - private final List segments; + private final Set oldSegments; + private final Set newSegments; @JsonCreator public PushedSegmentsReport( @JsonProperty("taskId") String taskId, - @JsonProperty("segments") List segments + @JsonProperty("oldSegments") Set oldSegments, + @JsonProperty("segments") Set newSegments ) { this.taskId = Preconditions.checkNotNull(taskId, "taskId"); - this.segments = Preconditions.checkNotNull(segments, "segments"); + this.oldSegments = Preconditions.checkNotNull(oldSegments, "oldSegments"); + this.newSegments = Preconditions.checkNotNull(newSegments, "newSegments"); } @JsonProperty @@ -53,8 +56,14 @@ public String getTaskId() } @JsonProperty - public List getSegments() + public Set getOldSegments() { - return segments; + return oldSegments; + } + + @JsonProperty("segments") + public Set getNewSegments() + { + return newSegments; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index ecbd8ca78800..4fd2de57d3fb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalOverwriteAction; import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorEntry; import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import org.apache.druid.java.util.common.ISE; @@ -47,6 +48,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -61,7 +63,7 @@ /** * An implementation of {@link ParallelIndexTaskRunner} to support best-effort roll-up. This runner can submit and * monitor multiple {@link ParallelIndexSubTask}s. - * + *

* As its name indicates, distributed indexing is done in a single phase, i.e., without shuffling intermediate data. As * a result, this task can't be used for perfect rollup. */ @@ -81,7 +83,9 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); - /** subTaskId -> report */ + /** + * subTaskId -> report + */ private final ConcurrentHashMap segmentsMap = new ConcurrentHashMap<>(); private volatile boolean subTaskScheduleAndMonitorStopped; @@ -115,7 +119,7 @@ public TaskState run() throws Exception log.warn("There's no input split to process"); return TaskState.SUCCESS; } - + final Iterator subTaskSpecIterator = subTaskSpecIterator().iterator(); final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs(); @@ -269,7 +273,7 @@ public void collectReport(PushedSegmentsReport report) segmentsMap.compute(report.getTaskId(), (taskId, prevReport) -> { if (prevReport != null) { Preconditions.checkState( - prevReport.getSegments().equals(report.getSegments()), + prevReport.getNewSegments().equals(report.getNewSegments()), "task[%s] sent two or more reports and previous report[%s] is different from the current one[%s]", taskId, prevReport, @@ -400,16 +404,25 @@ public TaskHistory getCompleteSubTaskSpecAttemptHistory(St private void publish(TaskToolbox toolbox) throws IOException { - final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { - final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments); - return toolbox.getTaskActionClient().submit(action); - }; final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()); - final Set segmentsToPublish = segmentsMap + final Set segmentsToBeOverwritten = new HashSet<>(); + final Set segmentsToPublish = new HashSet<>(); + segmentsMap .values() - .stream() - .flatMap(report -> report.getSegments().stream()) - .collect(Collectors.toSet()); + .forEach(report -> { + segmentsToBeOverwritten.addAll(report.getOldSegments()); + segmentsToPublish.addAll(report.getNewSegments()); + }); + final TransactionalSegmentPublisher publisher; + if (segmentsToBeOverwritten.isEmpty()) { + publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient().submit( + new SegmentTransactionalInsertAction(segments) + ); + } else { + publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient().submit( + new SegmentTransactionalOverwriteAction(segmentsToBeOverwritten, segments) + ); + } final boolean published = segmentsToPublish.isEmpty() || publisher.publishSegments(segmentsToPublish, null).isSuccess(); @@ -420,7 +433,7 @@ private void publish(TaskToolbox toolbox) throws IOException final Set segmentsIdentifiers = segmentsMap .values() .stream() - .flatMap(report -> report.getSegments().stream()) + .flatMap(report -> report.getNewSegments().stream()) .map(SegmentIdWithShardSpec::fromDataSegment) .collect(Collectors.toSet()); if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 15e9aa5e12ee..329028b33395 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -30,6 +30,7 @@ import org.apache.druid.indexing.common.actions.LocalTaskActionClient; import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalOverwriteAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionToolbox; import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; @@ -209,6 +210,8 @@ public RetType submit(TaskAction taskAction) publishedSegments.addAll(((SegmentTransactionalInsertAction) taskAction).getSegments()); } else if (taskAction instanceof SegmentInsertAction) { publishedSegments.addAll(((SegmentInsertAction) taskAction).getSegments()); + } else if (taskAction instanceof SegmentTransactionalOverwriteAction) { + publishedSegments.addAll(((SegmentTransactionalOverwriteAction) taskAction).getNewSegments()); } return result; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 30c932c06629..2bc229d43ad8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -68,8 +68,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; @@ -364,9 +364,9 @@ public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime } @Override - public void report(String supervisorTaskId, List pushedSegments) + public void report(String supervisorTaskId, Set oldSegments, Set pushedSegments) { - supervisorTask.getRunner().collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments)); + supervisorTask.getRunner().collectReport(new PushedSegmentsReport(getSubtaskId(), oldSegments, pushedSegments)); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index a04bdc4802c7..15a8fec548db 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -677,7 +677,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception 1L ); - taskClient.report(getId(), Collections.singletonList(segment)); + taskClient.report(getId(), Collections.emptySet(), Collections.singleton(segment)); return TaskStatus.fromCode(getId(), state); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index cde4679faa3b..fd3d127025ce 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -51,13 +51,11 @@ import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.stream.Collectors; public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest @@ -350,7 +348,6 @@ private ParallelIndexSupervisorTask newTask( new LongSumAggregatorFactory("val", "val") }, new UniformGranularitySpec( -// Granularities.DAY, segmentGranularity, Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java index 0673571ae380..713307254c9f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -48,10 +48,9 @@ public interface TransactionalSegmentPublisher * @throws IOException if there was an I/O error when publishing * @throws RuntimeException if we cannot tell if the segments were published or not, for some other reason */ - SegmentPublishResult publishAnnotatedSegments( - Set segments, - @Nullable Object commitMetadata - ) throws IOException; + // TODO: add oldSegments here?? + SegmentPublishResult publishAnnotatedSegments(Set segments, @Nullable Object commitMetadata) + throws IOException; default SegmentPublishResult publishSegments(Set segments, @Nullable Object commitMetadata) throws IOException From f0d23916386b683bab98b1394498800b97f31f47 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 3 May 2019 17:34:01 -0700 Subject: [PATCH 21/52] changing publisher --- .../kafka/LegacyKafkaIndexTaskRunner.java | 9 +- .../common/actions/SegmentInsertAction.java | 2 +- .../SegmentTransactionalInsertAction.java | 127 ++++++++-- .../SegmentTransactionalOverwriteAction.java | 218 ------------------ .../AppenderatorDriverRealtimeIndexTask.java | 11 +- .../druid/indexing/common/task/IndexTask.java | 25 +- .../SinglePhaseParallelIndexTaskRunner.java | 33 ++- .../seekablestream/SequenceMetadata.java | 11 +- .../SegmentTransactionalInsertActionTest.java | 11 +- .../common/task/IngestionTestBase.java | 3 - .../IndexerMetadataStorageCoordinator.java | 4 +- .../IndexerSQLMetadataStorageCoordinator.java | 4 +- .../appenderator/BaseAppenderatorDriver.java | 2 + .../appenderator/BatchAppenderatorDriver.java | 9 +- .../StreamAppenderatorDriver.java | 1 + .../TransactionalSegmentPublisher.java | 20 +- .../BatchAppenderatorDriverTest.java | 6 +- .../StreamAppenderatorDriverTest.java | 5 +- 18 files changed, 197 insertions(+), 304 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 28a152ad062a..854ef1b22371 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -522,7 +522,10 @@ public void run() status = Status.PUBLISHING; } - final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> { + final TransactionalSegmentPublisher publisher = (mustBeNullOrEmptySegments, segments, commitMetadata) -> { + if (mustBeNullOrEmptySegments != null && !mustBeNullOrEmptySegments.isEmpty()) { + throw new ISE("WTH? stream ingestion tasks are overwriting segments[%s]", mustBeNullOrEmptySegments); + } final SeekableStreamEndSequenceNumbers finalPartitions = toolbox.getObjectMapper().convertValue( ((Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata")).get(METADATA_NEXT_PARTITIONS), toolbox.getObjectMapper() @@ -543,13 +546,13 @@ public void run() final SegmentTransactionalInsertAction action; if (ioConfig.isUseTransaction()) { - action = new SegmentTransactionalInsertAction( + action = SegmentTransactionalInsertAction.appendAction( segments, new KafkaDataSourceMetadata(ioConfig.getStartSequenceNumbers()), new KafkaDataSourceMetadata(finalPartitions) ); } else { - action = new SegmentTransactionalInsertAction(segments, null, null); + action = SegmentTransactionalInsertAction.appendAction(segments, null, null); } log.info("Publishing with isTransaction[%s].", ioConfig.isUseTransaction()); 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 f424c7ce6f02..e7804564ef8a 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 TypeReference> getReturnTypeReference() @Override public Set 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/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java index 2af0899a34b0..269cf829e3fd 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,26 +22,31 @@ 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.SegmentLock; import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.IndexTaskUtils; 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; /** * Insert segments into metadata storage. The segment versions must all be less than or equal to a lock held by @@ -53,27 +58,51 @@ */ public class SegmentTransactionalInsertAction implements TaskAction { + @Nullable + private final Set segmentsToBeOverwritten; private final Set segments; + @Nullable private final DataSourceMetadata startMetadata; + @Nullable private final DataSourceMetadata endMetadata; - public SegmentTransactionalInsertAction(Set segments) + public static SegmentTransactionalInsertAction overwriteAction( + @Nullable Set segmentsToBeOverwritten, + Set segmentsToPublish + ) { - this(segments, null, null); + return new SegmentTransactionalInsertAction(segmentsToBeOverwritten, segmentsToPublish, null, null); + } + + public static SegmentTransactionalInsertAction appendAction( + Set segments, + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata + ) + { + return new SegmentTransactionalInsertAction(null, segments, startMetadata, endMetadata); } @JsonCreator - public SegmentTransactionalInsertAction( + private SegmentTransactionalInsertAction( + @JsonProperty("segmentsToBeOverwritten") Set segmentsToBeOverwritten, @JsonProperty("segments") Set 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 + public Set getSegmentsToBeOverwritten() + { + return segmentsToBeOverwritten; + } + @JsonProperty public Set getSegments() { @@ -81,12 +110,14 @@ public Set getSegments() } @JsonProperty + @Nullable public DataSourceMetadata getStartMetadata() { return startMetadata; } @JsonProperty + @Nullable public DataSourceMetadata getEndMetadata() { return endMetadata; @@ -101,25 +132,30 @@ public TypeReference getReturnTypeReference() } /** - * 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 allSegments = new HashSet<>(segments); + if (segmentsToBeOverwritten != null) { + allSegments.addAll(segmentsToBeOverwritten); + } + TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); - final Map> intervalToPartitionIds = new HashMap<>(); - for (DataSegment segment : segments) { - intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new HashSet<>()) - .add(segment.getShardSpec().getPartitionNum()); + if (segmentsToBeOverwritten != null && !segmentsToBeOverwritten.isEmpty()) { + final List 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, - new ArrayList<>(intervalToPartitionIds.keySet()), + allSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()), CriticalAction.builder() .onValidLocks( () -> toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments( @@ -150,7 +186,6 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) } } - // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); @@ -170,6 +205,67 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) return retVal; } + private void checkWithSegmentLock() + { + final Map> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten); + final Map> newSegmentsMap = groupSegmentsByIntervalAndSort(segments); + + oldSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); + newSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); + + oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> { + final List 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> groupSegmentsByIntervalAndSort(Set segments) + { + final Map> 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() { @@ -180,7 +276,8 @@ public boolean isAudited() 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/SegmentTransactionalOverwriteAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java deleted file mode 100644 index 273c9acbd10c..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalOverwriteAction.java +++ /dev/null @@ -1,218 +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.fasterxml.jackson.annotation.JsonCreator; -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.LockGranularity; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; -import org.apache.druid.indexing.common.task.IndexTaskUtils; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.CriticalAction; -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 java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -// TODO: merging with SegmentTransactionalInsertAction? -public class SegmentTransactionalOverwriteAction implements TaskAction -{ - private final Set oldSegments; - private final Set newSegments; - - @JsonCreator - public SegmentTransactionalOverwriteAction( - @JsonProperty("oldSegments") Set oldSegments, - @JsonProperty("newSegments") Set newSegments - ) - { - this.oldSegments = oldSegments; - this.newSegments = newSegments; - } - - @JsonProperty - public Set getOldSegments() - { - return oldSegments; - } - - @JsonProperty - public Set getNewSegments() - { - return newSegments; - } - - @Override - public TypeReference getReturnTypeReference() - { - return new TypeReference() - { - }; - } - - @Override - public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) - { - final Set allSegments = new HashSet<>(oldSegments); - allSegments.addAll(newSegments); - TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); - - final Map> intervalToPartitionIds = new HashMap<>(); - for (DataSegment segment : allSegments) { - intervalToPartitionIds.computeIfAbsent(segment.getInterval(), k -> new HashSet<>()) - .add(segment.getShardSpec().getPartitionNum()); - } - - final List 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, - new ArrayList<>(intervalToPartitionIds.keySet()), - CriticalAction.builder() - .onValidLocks( - () -> toolbox.getIndexerMetadataStorageCoordinator() - .announceHistoricalSegments(newSegments, null, null) - ) - .onInvalidLocks( - () -> SegmentPublishResult.fail( - "Invalid task locks. Maybe they are revoked by a higher priority task." - + " Please check the overlord log for details." - ) - ) - .build() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - - // Emit metrics - final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); - IndexTaskUtils.setTaskDimensions(metricBuilder, task); - - if (retVal.isSuccess()) { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/success", 1)); - } else { - toolbox.getEmitter().emit(metricBuilder.build("segment/txn/failure", 1)); - } - - // getSegments() should return an empty set if announceHistoricalSegments() failed - for (DataSegment segment : retVal.getSegments()) { - metricBuilder.setDimension(DruidMetrics.INTERVAL, segment.getInterval().toString()); - toolbox.getEmitter().emit(metricBuilder.build("segment/added/bytes", segment.getSize())); - } - - return retVal; - } - - private void checkWithSegmentLock() - { - final Map> oldSegmentsMap = groupSegmentsByIntervalAndSort(oldSegments); - final Map> newSegmentsMap = groupSegmentsByIntervalAndSort(newSegments); - - oldSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); - newSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); - - oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> { - final List 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> groupSegmentsByIntervalAndSort(Set segments) - { - final Map> 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() - { - return false; - } - - @Override - public String toString() - { - return "SegmentTransactionalOverwriteAction{" + - "oldSegments=" + oldSegments + - ", newSegments=" + newSegments + - '}'; - } -} 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 004a6fb5b7b6..8e6834ffb0be 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 @@ -310,8 +310,15 @@ public TaskStatus run(final TaskToolbox toolbox) 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); }; 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 9e69f618e24d..4eb545afc8c9 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 @@ -47,7 +47,6 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalOverwriteAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -952,19 +951,9 @@ private TaskStatus generateAndPublishSegments( allocateSpec ); - final TransactionalSegmentPublisher publisher; - if (!isGuaranteedRollup && isOverwriteMode() && !isChangeSegmentGranularity()) { - publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient() - .submit( - new SegmentTransactionalOverwriteAction( - getAllInputSegments(), - segments - ) - ); - } else { - publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient() - .submit(new SegmentTransactionalInsertAction(segments)); - } + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient() + .submit(SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish)); try ( final Appenderator appenderator = newAppenderator( @@ -1042,10 +1031,10 @@ private TaskStatus generateAndPublishSegments( log.info("Pushed segments[%s]", pushed.getSegments()); // Probably we can publish atomicUpdateGroup along with segments. - final SegmentsAndMetadata published = awaitPublish( - driver.publishAll(publisher), - pushTimeout - ); + final Set inputSegments = !isGuaranteedRollup && isOverwriteMode() && !isChangeSegmentGranularity() + ? getAllInputSegments() + : null; + final SegmentsAndMetadata published = awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout); ingestionState = IngestionState.COMPLETED; if (published == null) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 4fd2de57d3fb..79648a3f6301 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalOverwriteAction; import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorEntry; import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; import org.apache.druid.java.util.common.ISE; @@ -405,29 +404,23 @@ public TaskHistory getCompleteSubTaskSpecAttemptHistory(St private void publish(TaskToolbox toolbox) throws IOException { final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()); - final Set segmentsToBeOverwritten = new HashSet<>(); - final Set segmentsToPublish = new HashSet<>(); + final Set oldSegments = new HashSet<>(); + final Set newSegments = new HashSet<>(); segmentsMap .values() .forEach(report -> { - segmentsToBeOverwritten.addAll(report.getOldSegments()); - segmentsToPublish.addAll(report.getNewSegments()); + oldSegments.addAll(report.getOldSegments()); + newSegments.addAll(report.getNewSegments()); }); - final TransactionalSegmentPublisher publisher; - if (segmentsToBeOverwritten.isEmpty()) { - publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient().submit( - new SegmentTransactionalInsertAction(segments) - ); - } else { - publisher = (segments, commitMetadata) -> toolbox.getTaskActionClient().submit( - new SegmentTransactionalOverwriteAction(segmentsToBeOverwritten, segments) - ); - } - final boolean published = segmentsToPublish.isEmpty() - || publisher.publishSegments(segmentsToPublish, null).isSuccess(); + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit( + SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish) + ); + final boolean published = newSegments.isEmpty() + || publisher.publishSegments(oldSegments, newSegments, null).isSuccess(); if (published) { - log.info("Published [%d] segments", segmentsToPublish.size()); + log.info("Published [%d] segments", newSegments.size()); } else { log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); final Set segmentsIdentifiers = segmentsMap @@ -437,10 +430,10 @@ private void publish(TaskToolbox toolbox) throws IOException .map(SegmentIdWithShardSpec::fromDataSegment) .collect(Collectors.toSet()); if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers) - .equals(segmentsToPublish)) { + .equals(newSegments)) { log.info("Our segments really do exist, awaiting handoff."); } else { - throw new ISE("Failed to publish segments[%s]", segmentsToPublish); + throw new ISE("Failed to publish segments[%s]", newSegments); } } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java index 20eec3591ba9..17ea3c9415b4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SequenceMetadata.java @@ -301,7 +301,10 @@ TransactionalSegmentPublisher createPublisher( boolean useTransaction ) { - return (segments, commitMetadata) -> { + return (mustBeNullOrEmptySegments, segmentsToPush, commitMetadata) -> { + if (mustBeNullOrEmptySegments != null && !mustBeNullOrEmptySegments.isEmpty()) { + throw new ISE("WTH? stream ingestion tasks are overwriting segments[%s]", mustBeNullOrEmptySegments); + } final Map commitMetaMap = (Map) Preconditions.checkNotNull(commitMetadata, "commitMetadata"); final SeekableStreamEndSequenceNumbers finalPartitions = runner.deserializePartitionsFromMetadata( @@ -321,8 +324,8 @@ TransactionalSegmentPublisher createPublisher( final SegmentTransactionalInsertAction action; if (useTransaction) { - action = new SegmentTransactionalInsertAction( - segments, + action = SegmentTransactionalInsertAction.appendAction( + segmentsToPush, runner.createDataSourceMetadata( new SeekableStreamStartSequenceNumbers<>( finalPartitions.getStream(), @@ -333,7 +336,7 @@ TransactionalSegmentPublisher createPublisher( runner.createDataSourceMetadata(finalPartitions) ); } else { - action = new SegmentTransactionalInsertAction(segments, null, null); + action = SegmentTransactionalInsertAction.appendAction(segmentsToPush, null, null); } return toolbox.getTaskActionClient().submit(action); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java index ed08c4fcb82c..42417d680b06 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java @@ -101,7 +101,7 @@ public void testTransactional() throws Exception actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - SegmentPublishResult result1 = new SegmentTransactionalInsertAction( + SegmentPublishResult result1 = SegmentTransactionalInsertAction.appendAction( ImmutableSet.of(SEGMENT1), new ObjectMetadata(null), new ObjectMetadata(ImmutableList.of(1)) @@ -111,7 +111,7 @@ public void testTransactional() throws Exception ); Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(SEGMENT1)), result1); - SegmentPublishResult result2 = new SegmentTransactionalInsertAction( + SegmentPublishResult result2 = SegmentTransactionalInsertAction.appendAction( ImmutableSet.of(SEGMENT2), new ObjectMetadata(ImmutableList.of(1)), new ObjectMetadata(ImmutableList.of(2)) @@ -142,7 +142,7 @@ public void testFailTransactional() throws Exception actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); - SegmentPublishResult result = new SegmentTransactionalInsertAction( + SegmentPublishResult result = SegmentTransactionalInsertAction.appendAction( ImmutableSet.of(SEGMENT1), new ObjectMetadata(ImmutableList.of(1)), new ObjectMetadata(ImmutableList.of(2)) @@ -158,7 +158,10 @@ public void testFailTransactional() throws Exception public void testFailBadVersion() throws Exception { final Task task = NoopTask.create(); - final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(ImmutableSet.of(SEGMENT3)); + final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.overwriteAction( + null, + ImmutableSet.of(SEGMENT3) + ); actionTestKit.getTaskLockbox().add(task); acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 329028b33395..15e9aa5e12ee 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -30,7 +30,6 @@ import org.apache.druid.indexing.common.actions.LocalTaskActionClient; import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.actions.SegmentTransactionalOverwriteAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionToolbox; import org.apache.druid.indexing.common.actions.TaskAuditLogConfig; @@ -210,8 +209,6 @@ public RetType submit(TaskAction taskAction) publishedSegments.addAll(((SegmentTransactionalInsertAction) taskAction).getSegments()); } else if (taskAction instanceof SegmentInsertAction) { publishedSegments.addAll(((SegmentInsertAction) taskAction).getSegments()); - } else if (taskAction instanceof SegmentTransactionalOverwriteAction) { - publishedSegments.addAll(((SegmentTransactionalOverwriteAction) taskAction).getNewSegments()); } return result; } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 979a660bf012..ead6498fbdbe 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -147,8 +147,8 @@ SegmentIdWithShardSpec allocatePendingSegment( */ SegmentPublishResult announceHistoricalSegments( Set segments, - DataSourceMetadata startMetadata, - DataSourceMetadata endMetadata + @Nullable DataSourceMetadata startMetadata, + @Nullable DataSourceMetadata endMetadata ) throws IOException; /** diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 20b146bce1be..7c435968c08f 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -250,8 +250,8 @@ public Set announceHistoricalSegments(final Set segmen @Override public SegmentPublishResult announceHistoricalSegments( final Set segments, - final DataSourceMetadata startMetadata, - final DataSourceMetadata endMetadata + @Nullable final DataSourceMetadata startMetadata, + @Nullable final DataSourceMetadata endMetadata ) throws IOException { if (segments.isEmpty()) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index 46c06eeef2fd..25117b36604a 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -538,6 +538,7 @@ ListenableFuture dropInBackground(SegmentsAndMetadata segme * @return a future for publishing segments */ ListenableFuture publishInBackground( + @Nullable Set segmentsToBeOverwritten, SegmentsAndMetadata segmentsAndMetadata, TransactionalSegmentPublisher publisher ) @@ -557,6 +558,7 @@ ListenableFuture publishInBackground( final Object metadata = segmentsAndMetadata.getCommitMetadata(); final ImmutableSet ourSegments = ImmutableSet.copyOf(segmentsAndMetadata.getSegments()); final SegmentPublishResult publishResult = publisher.publishSegments( + segmentsToBeOverwritten, ourSegments, metadata == null ? null : ((AppenderatorDriverMetadata) metadata).getCallerMetadata() ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java index c38f853ef7d0..8b6c7c536a8f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java @@ -186,11 +186,15 @@ private SegmentsAndMetadata pushAndClear( /** * Publish all segments. * - * @param publisher segment publisher + * @param segmentsToBeOverwritten segments which can be overwritten by new segments published by the given publisher + * @param publisher segment publisher * * @return a {@link ListenableFuture} for the publish task */ - public ListenableFuture publishAll(final TransactionalSegmentPublisher publisher) + public ListenableFuture publishAll( + @Nullable final Set segmentsToBeOverwritten, + final TransactionalSegmentPublisher publisher + ) { final Map snapshot; synchronized (segments) { @@ -198,6 +202,7 @@ public ListenableFuture publishAll(final TransactionalSegme } return publishInBackground( + segmentsToBeOverwritten, new SegmentsAndMetadata( snapshot .values() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index ea98d87d1f68..b589b69e7590 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -281,6 +281,7 @@ public ListenableFuture publish( // version of a segment with the same identifier containing different data; see DataSegmentPusher.push() docs pushInBackground(wrapCommitter(committer), theSegments, true), sam -> publishInBackground( + null, sam, publisher ) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java index 713307254c9f..8ebc66237935 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -48,14 +48,24 @@ public interface TransactionalSegmentPublisher * @throws IOException if there was an I/O error when publishing * @throws RuntimeException if we cannot tell if the segments were published or not, for some other reason */ - // TODO: add oldSegments here?? - SegmentPublishResult publishAnnotatedSegments(Set segments, @Nullable Object commitMetadata) - throws IOException; + SegmentPublishResult publishAnnotatedSegments( + @Nullable Set segmentsToBeOverwritten, + Set segmentsToPublish, + @Nullable Object commitMetadata + ) throws IOException; - default SegmentPublishResult publishSegments(Set segments, @Nullable Object commitMetadata) + default SegmentPublishResult publishSegments( + @Nullable Set segmentsToBeOverwritten, + Set segmentsToPublish, + @Nullable Object commitMetadata + ) throws IOException { - return publishAnnotatedSegments(annotateAtomicUpdateGroupSize(segments), commitMetadata); + return publishAnnotatedSegments( + segmentsToBeOverwritten, + annotateAtomicUpdateGroupSize(segmentsToPublish), + commitMetadata + ); } static Set annotateAtomicUpdateGroupSize(Set segments) diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java index a484ba2117cf..f01495d469e7 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriverTest.java @@ -115,7 +115,7 @@ public void testSimple() throws Exception checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED); - final SegmentsAndMetadata published = driver.publishAll(makeOkPublisher()).get( + final SegmentsAndMetadata published = driver.publishAll(null, makeOkPublisher()).get( TIMEOUT, TimeUnit.MILLISECONDS ); @@ -151,7 +151,7 @@ public void testIncrementalPush() throws Exception checkSegmentStates(++i, SegmentState.PUSHED_AND_DROPPED); } - final SegmentsAndMetadata published = driver.publishAll(makeOkPublisher()).get( + final SegmentsAndMetadata published = driver.publishAll(null, makeOkPublisher()).get( TIMEOUT, TimeUnit.MILLISECONDS ); @@ -195,6 +195,6 @@ private void checkSegmentStates(int expectedNumSegmentsInState, SegmentState exp static TransactionalSegmentPublisher makeOkPublisher() { - return (segments, commitMetadata) -> SegmentPublishResult.ok(ImmutableSet.of()); + return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> SegmentPublishResult.ok(ImmutableSet.of()); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java index c544eb6c8314..c474430e4308 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverTest.java @@ -361,12 +361,13 @@ private Set asIdentifiers(Iterable segments static TransactionalSegmentPublisher makeOkPublisher() { - return (segments, commitMetadata) -> SegmentPublishResult.ok(Collections.emptySet()); + return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + SegmentPublishResult.ok(Collections.emptySet()); } static TransactionalSegmentPublisher makeFailingPublisher(boolean failWithException) { - return (segments, commitMetadata) -> { + return (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> { final RuntimeException exception = new RuntimeException("test"); if (failWithException) { throw exception; From c24d2356db01134e84293062bab219c74d95064e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 6 May 2019 16:58:41 -0700 Subject: [PATCH 22/52] fixing something --- .../common/task/AbstractBatchIndexTask.java | 26 +++++++++++-------- .../task/CachingLocalSegmentAllocator.java | 1 + .../druid/indexing/common/task/IndexTask.java | 7 +++-- .../common/task/RemoteSegmentAllocator.java | 13 +++++----- .../batch/parallel/ParallelIndexSubTask.java | 4 +-- 5 files changed, 27 insertions(+), 24 deletions(-) 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 index 05dc23e34a68..4b72a5e35cb1 100644 --- 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 @@ -291,12 +291,6 @@ private void verifyAndFindRootPartitionRangeAndMinorVersion(List in ); } - // TODO: perhaps merge the below methods - protected boolean isChangeSegmentGranularity() - { - return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); - } - public Set getAllInputSegments() { return Preconditions.checkNotNull(allInputSegments, "allInputSegments is not initialized"); @@ -308,19 +302,29 @@ Map getAllOverwritingSegmentMeta( return Collections.unmodifiableMap(overwritingRootGenPartitions); } - @Nullable - public OverwritingRootGenerationPartitions getOverwritingSegmentMeta(Interval interval) + public boolean needMinorOverwrite() { - Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); - return overwritingRootGenPartitions.get(interval); + return hasInputSegments() && !isChangeSegmentGranularity(); } - public boolean isOverwriteMode() + public boolean isChangeSegmentGranularity() + { + return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); + } + + private boolean hasInputSegments() { Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); return !overwritingRootGenPartitions.isEmpty(); } + @Nullable + public OverwritingRootGenerationPartitions getOverwritingSegmentMeta(Interval interval) + { + Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); + return overwritingRootGenPartitions.get(interval); + } + public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List sortedSegments) { if (sortedSegments.isEmpty()) { 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 index 5bef3db5c0fb..1d86733b7283 100644 --- 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 @@ -61,6 +61,7 @@ public CachingLocalSegmentAllocator( .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); } + // TODO: fix - this is called before intervalToVersion is initialized @Override Map> getIntervalToSegmentIds() { 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 4eb545afc8c9..f1ca0c5984cc 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 @@ -874,7 +874,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( Map> allocateSpec ) throws IOException { - if (ingestionSchema.ioConfig.isAppendToExisting() || !isChangeSegmentGranularity()) { + if (!isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig) && (ingestionSchema.ioConfig.isAppendToExisting() || !isChangeSegmentGranularity())) { if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { return new CachingRemoteSegmentAllocator(toolbox, getId(), allocateSpec); } else { @@ -882,8 +882,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( toolbox, getId(), dataSchema, - isOverwriteMode(), - isChangeSegmentGranularity(), + needMinorOverwrite(), getAllOverwritingSegmentMeta() ); } @@ -1031,7 +1030,7 @@ private TaskStatus generateAndPublishSegments( log.info("Pushed segments[%s]", pushed.getSegments()); // Probably we can publish atomicUpdateGroup along with segments. - final Set inputSegments = !isGuaranteedRollup && isOverwriteMode() && !isChangeSegmentGranularity() + final Set inputSegments = !isGuaranteedRollup && needMinorOverwrite() ? getAllInputSegments() : null; final SegmentsAndMetadata published = awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java index a0269f6cc520..20e45c8957c2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java @@ -42,12 +42,11 @@ public class RemoteSegmentAllocator implements IndexTaskSegmentAllocator private final ActionBasedSegmentAllocator internalAllocator; public RemoteSegmentAllocator( - TaskToolbox toolbox, - String taskId, - DataSchema dataSchema, - boolean isOverwriteMode, - boolean isChangeSegmentGranularity, - Map overwritingSegmentMetaMap + final TaskToolbox toolbox, + final String taskId, + final DataSchema dataSchema, + final boolean needMinorOverwrite, + final Map overwritingSegmentMetaMap ) { this.taskId = taskId; @@ -60,7 +59,7 @@ public RemoteSegmentAllocator( .bucketInterval(row.getTimestamp()) .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); final ShardSpecFactory shardSpecFactory; - if (isOverwriteMode && !isChangeSegmentGranularity) { + if (needMinorOverwrite) { final OverwritingRootGenerationPartitions overwritingSegmentMeta = overwritingSegmentMetaMap.get(interval); if (overwritingSegmentMeta == null) { throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 24b71dc64e82..561726ca5607 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -303,7 +303,7 @@ public SegmentIdWithShardSpec allocate( private SegmentAllocator createSegmentAllocator() { - if (ingestionSchema.getIOConfig().isAppendToExisting() || !isChangeSegmentGranularity()) { + if (ingestionSchema.getIOConfig().isAppendToExisting() || needMinorOverwrite()) { return new ActionBasedSegmentAllocator( toolbox.getTaskActionClient(), ingestionSchema.getDataSchema(), @@ -313,7 +313,7 @@ private SegmentAllocator createSegmentAllocator() .bucketInterval(row.getTimestamp()) .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); final ShardSpecFactory shardSpecFactory; - if (isOverwriteMode() && !isChangeSegmentGranularity()) { + if (needMinorOverwrite()) { final OverwritingRootGenerationPartitions overwritingSegmentMeta = getOverwritingSegmentMeta(interval); if (overwritingSegmentMeta == null) { throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); From 2ea0b34c7deb228771ec30a8108cd280b03fb1ef Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 May 2019 17:45:54 -0700 Subject: [PATCH 23/52] fix for perfect rollup --- .../common/task/AbstractBatchIndexTask.java | 63 ++++++++++--------- .../task/CachingLocalSegmentAllocator.java | 25 +++----- .../task/CachingRemoteSegmentAllocator.java | 3 +- .../common/task/CachingSegmentAllocator.java | 8 +++ .../indexing/common/task/HadoopIndexTask.java | 9 ++- .../druid/indexing/common/task/IndexTask.java | 10 ++- .../batch/parallel/ParallelIndexSubTask.java | 8 ++- .../parallel/ParallelIndexSupervisorTask.java | 8 ++- 8 files changed, 84 insertions(+), 50 deletions(-) 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 index 4b72a5e35cb1..76ba16251d3b 100644 --- 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 @@ -130,13 +130,15 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i throws IOException { if (requireLockInputSegments()) { - // This method finds segments falling in all given intervals and then tries to lock those segments. - // Thus, there might be a race between calling findInputSegments() and tryLockWithSegments(), - // 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 findInputSegments() call, it may missing those - // segments. This is usually fine, but if you want to avoid this, you should use timeChunk lock instead. - if (!intervals.isEmpty()) { + if (isPerfectRollup()) { + return tryTimeChunkLock(client, intervals); + } 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 findInputSegments() and tryLockWithSegments(), + // 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 findInputSegments() call, it may missing those + // segments. This is usually fine, but if you want to avoid this, you should use timeChunk lock instead. return tryLockWithSegments(client, findInputSegments(client, intervals), isInitialRequest); } else { return true; @@ -149,6 +151,30 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } + boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException + { + allInputSegments = Collections.emptySet(); + overwritingRootGenPartitions = Collections.emptyMap(); + // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined + final Set uniqueIntervals = new HashSet<>(); + for (Interval interval : JodaUtils.condenseIntervals(intervals)) { + final Granularity segmentGranularity = getSegmentGranularity(interval); + 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; + } + boolean tryLockWithSegments(TaskActionClient client, List segments, boolean isInitialRequest) throws IOException { if (segments.isEmpty()) { @@ -163,27 +189,8 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); changeSegmentGranularity = changeSegmentGranularity(intervals); - if (changeSegmentGranularity || isPerfectRollup()) { - allInputSegments = Collections.emptySet(); - overwritingRootGenPartitions = Collections.emptyMap(); - // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined - final Set uniqueIntervals = new HashSet<>(); - for (Interval interval : JodaUtils.condenseIntervals(intervals)) { - final Granularity segmentGranularity = getSegmentGranularity(interval); - 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; + if (changeSegmentGranularity) { + return tryTimeChunkLock(client, intervals); } else { final List segmentsToLock; final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( 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 index 1d86733b7283..9e5a284eb75d 100644 --- 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 @@ -39,10 +39,6 @@ public class CachingLocalSegmentAllocator extends CachingSegmentAllocator { - private final String dataSource; - - private final Map intervalToVersion; - public CachingLocalSegmentAllocator( TaskToolbox toolbox, String taskId, @@ -52,19 +48,16 @@ public CachingLocalSegmentAllocator( { // This segment allocator doesn't need inputPartitionIds because the newly created segments don't have to store // direcOvershadowingSegments - super(toolbox, taskId, allocateSpec); - this.dataSource = dataSource; - - intervalToVersion = toolbox.getTaskActionClient() - .submit(new LockListAction()) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + super(toolbox, taskId, dataSource, allocateSpec); } - // TODO: fix - this is called before intervalToVersion is initialized @Override - Map> getIntervalToSegmentIds() + Map> getIntervalToSegmentIds() throws IOException { + final Map intervalToVersion = getToolbox().getTaskActionClient() + .submit(new LockListAction()) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); final Map> allocateSpec = getAllocateSpec(); final Map> intervalToSegmentIds = new HashMap<>(allocateSpec.size()); for (Entry> entry : allocateSpec.entrySet()) { @@ -80,9 +73,9 @@ Map> getIntervalToSegmentIds() interval, IntStream.range(0, numSegmentsToAllocate) .mapToObj(i -> new SegmentIdWithShardSpec( - dataSource, + getDataSource(), interval, - findVersion(interval), + findVersion(intervalToVersion, interval), shardSpecFactory.create(getToolbox().getObjectMapper(), i) )) .collect(Collectors.toList()) @@ -91,7 +84,7 @@ Map> getIntervalToSegmentIds() return intervalToSegmentIds; } - private String findVersion(Interval interval) + private static String findVersion(Map intervalToVersion, Interval interval) { return intervalToVersion.entrySet().stream() .filter(entry -> entry.getKey().contains(interval)) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java index cff358c49788..5e96ddc6f3b4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java @@ -35,10 +35,11 @@ public class CachingRemoteSegmentAllocator extends CachingSegmentAllocator public CachingRemoteSegmentAllocator( TaskToolbox toolbox, String taskId, + String dataSource, Map> allocateSpec ) throws IOException { - super(toolbox, taskId, allocateSpec); + super(toolbox, taskId, dataSource, allocateSpec); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java index 8b7e12b092a7..708c513eab72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java @@ -40,6 +40,7 @@ public abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocat { private final TaskToolbox toolbox; private final String taskId; + private final String dataSource; private final Map> allocateSpec; @Nullable private final ShardSpecs shardSpecs; @@ -50,11 +51,13 @@ public abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocat public CachingSegmentAllocator( TaskToolbox toolbox, String taskId, + String dataSource, Map> allocateSpec ) throws IOException { this.toolbox = toolbox; this.taskId = taskId; + this.dataSource = dataSource; this.allocateSpec = allocateSpec; this.sequenceNameToSegmentId = new HashMap<>(); @@ -86,6 +89,11 @@ String getTaskId() return taskId; } + String getDataSource() + { + return dataSource; + } + Map> getAllocateSpec() { return allocateSpec; 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 4d9ae1b0a12d..6847f99f1d01 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 @@ -56,6 +56,8 @@ 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; @@ -235,7 +237,12 @@ public boolean isPerfectRollup() @Override public Granularity getSegmentGranularity(Interval interval) { - return spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); + final GranularitySpec granularitySpec = spec.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } } @JsonProperty("spec") 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 f1ca0c5984cc..ae0111e7986e 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 @@ -68,6 +68,7 @@ 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; @@ -335,7 +336,12 @@ public boolean isPerfectRollup() @Override public Granularity getSegmentGranularity(Interval interval) { - return ingestionSchema.dataSchema.getGranularitySpec().getSegmentGranularity(); + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } } private boolean tryLockIfNecessary(TaskActionClient actionClient, Collection intervals) throws IOException @@ -876,7 +882,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( { if (!isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig) && (ingestionSchema.ioConfig.isAppendToExisting() || !isChangeSegmentGranularity())) { if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { - return new CachingRemoteSegmentAllocator(toolbox, getId(), allocateSpec); + return new CachingRemoteSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); } else { return new RemoteSegmentAllocator( toolbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 561726ca5607..6c94ff2ef8c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -52,6 +52,7 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; +import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireDepartment; @@ -260,7 +261,12 @@ public boolean isPerfectRollup() @Override public Granularity getSegmentGranularity(Interval interval) { - return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } } @VisibleForTesting diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 9377e8b65f7d..fa3d728af85a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -51,6 +51,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; 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.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -258,7 +259,12 @@ public boolean isPerfectRollup() @Override public Granularity getSegmentGranularity(Interval interval) { - return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } } @Override From d2dbf5285874850cbdbdb88ecd2401bba2f020e5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 9 May 2019 16:36:25 -0700 Subject: [PATCH 24/52] fix test --- .../common/task/AbstractBatchIndexTask.java | 14 ++++------- .../indexing/common/task/CompactionTask.java | 2 +- .../indexing/common/task/HadoopIndexTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 25 ++++++++----------- .../batch/parallel/ParallelIndexSubTask.java | 6 ++--- .../parallel/ParallelIndexSupervisorTask.java | 2 +- .../indexing/common/task/HadoopTaskTest.java | 2 +- 7 files changed, 23 insertions(+), 30 deletions(-) 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 index 76ba16251d3b..3d1bb0acb712 100644 --- 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 @@ -112,7 +112,7 @@ protected AbstractBatchIndexTask( public abstract List findInputSegments(TaskActionClient taskActionClient, List intervals) throws IOException; - public abstract boolean changeSegmentGranularity(List intervalOfExistingSegments); + public abstract boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments); public abstract boolean isPerfectRollup(); @@ -175,7 +175,8 @@ boolean tryTimeChunkLock(TaskActionClient client, List intervals) thro return true; } - boolean tryLockWithSegments(TaskActionClient client, List segments, boolean isInitialRequest) throws IOException + boolean tryLockWithSegments(TaskActionClient client, List segments, boolean isInitialRequest) + throws IOException { if (segments.isEmpty()) { changeSegmentGranularity = false; @@ -188,7 +189,7 @@ boolean tryLockWithSegments(TaskActionClient client, List segments, // Create a timeline to find latest segments only final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); - changeSegmentGranularity = changeSegmentGranularity(intervals); + changeSegmentGranularity = checkIfChangeSegmentGranularity(intervals); if (changeSegmentGranularity) { return tryTimeChunkLock(client, intervals); } else { @@ -309,17 +310,12 @@ Map getAllOverwritingSegmentMeta( return Collections.unmodifiableMap(overwritingRootGenPartitions); } - public boolean needMinorOverwrite() - { - return hasInputSegments() && !isChangeSegmentGranularity(); - } - public boolean isChangeSegmentGranularity() { return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); } - private boolean hasInputSegments() + public boolean hasInputSegments() { Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); return !overwritingRootGenPartitions.isEmpty(); 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 a009280f65e7..687c9523d38a 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 @@ -292,7 +292,7 @@ public List findInputSegments(TaskActionClient taskActionClient, Li } @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) + public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) { return (keepSegmentGranularity != null && !keepSegmentGranularity) || (segmentGranularity != null); 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 6847f99f1d01..0efa7359ccac 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 @@ -221,7 +221,7 @@ public List findInputSegments(TaskActionClient taskActionClient, Li } @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) + public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) { final Granularity segmentGranularity = spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); 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 ae0111e7986e..ec7e4cf1ba05 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 @@ -320,7 +320,7 @@ public static List findInputSegments( } @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) + public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) { final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); @@ -880,18 +880,15 @@ private IndexTaskSegmentAllocator createSegmentAllocator( Map> allocateSpec ) throws IOException { - if (!isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig) && (ingestionSchema.ioConfig.isAppendToExisting() || !isChangeSegmentGranularity())) { - if (isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig)) { - return new CachingRemoteSegmentAllocator(toolbox, getId(), getDataSource(), allocateSpec); - } else { - return new RemoteSegmentAllocator( - toolbox, - getId(), - dataSchema, - needMinorOverwrite(), - getAllOverwritingSegmentMeta() - ); - } + if (!isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig) + && (ingestionSchema.ioConfig.isAppendToExisting() || !isChangeSegmentGranularity())) { + return new RemoteSegmentAllocator( + toolbox, + getId(), + dataSchema, + hasInputSegments() && !isChangeSegmentGranularity(), + getAllOverwritingSegmentMeta() + ); } else { // We use the timeChunk lock and don't have to ask the overlord to create segmentIds. // Instead, a local allocator is used. @@ -1036,7 +1033,7 @@ private TaskStatus generateAndPublishSegments( log.info("Pushed segments[%s]", pushed.getSegments()); // Probably we can publish atomicUpdateGroup along with segments. - final Set inputSegments = !isGuaranteedRollup && needMinorOverwrite() + final Set inputSegments = !isGuaranteedRollup && hasInputSegments() && !isChangeSegmentGranularity() ? getAllInputSegments() : null; final SegmentsAndMetadata published = awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 6c94ff2ef8c1..0252663b820e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -245,7 +245,7 @@ public List findInputSegments(TaskActionClient taskActionClient, Li } @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) + public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) { final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); @@ -309,7 +309,7 @@ public SegmentIdWithShardSpec allocate( private SegmentAllocator createSegmentAllocator() { - if (ingestionSchema.getIOConfig().isAppendToExisting() || needMinorOverwrite()) { + if (ingestionSchema.getIOConfig().isAppendToExisting() || !isChangeSegmentGranularity()) { return new ActionBasedSegmentAllocator( toolbox.getTaskActionClient(), ingestionSchema.getDataSchema(), @@ -319,7 +319,7 @@ private SegmentAllocator createSegmentAllocator() .bucketInterval(row.getTimestamp()) .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); final ShardSpecFactory shardSpecFactory; - if (needMinorOverwrite()) { + if (hasInputSegments() && !isChangeSegmentGranularity()) { final OverwritingRootGenerationPartitions overwritingSegmentMeta = getOverwritingSegmentMeta(interval); if (overwritingSegmentMeta == null) { throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index fa3d728af85a..e040eff95645 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -243,7 +243,7 @@ public boolean requireLockInputSegments() } @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) + public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) { final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java index 54c07876bec2..12e1cfffbec5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java @@ -82,7 +82,7 @@ public List findInputSegments(TaskActionClient taskActionClient, Li } @Override - public boolean changeSegmentGranularity(List intervalOfExistingSegments) + public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) { return true; } From 20eb5e6311d66c167166d0b67746b0e48634cdd3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 9 May 2019 17:07:25 -0700 Subject: [PATCH 25/52] adjust package-lock.json --- web-console/package-lock.json | 1578 ++++++++++++++++----------------- 1 file changed, 754 insertions(+), 824 deletions(-) diff --git a/web-console/package-lock.json b/web-console/package-lock.json index d3abb423c53d..e3ee01837e6a 100644 --- a/web-console/package-lock.json +++ b/web-console/package-lock.json @@ -14,18 +14,18 @@ } }, "@babel/core": { - "version": "7.3.4", - "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.3.4.tgz", - "integrity": "sha512-jRsuseXBo9pN197KnDwhhaaBzyZr2oIcLHHTt2oDdQrej5Qp57dCCJafWx5ivU8/alEYDpssYqv1MUqcxwQlrA==", + "version": "7.4.3", + "resolved": "https://registry.npmjs.org/@babel/core/-/core-7.4.3.tgz", + "integrity": "sha512-oDpASqKFlbspQfzAE7yaeTmdljSH2ADIvBlb0RwbStltTuWa0+7CCI1fYVINNv9saHPa1W7oaKeuNuKj+RQCvA==", "dev": true, "requires": { "@babel/code-frame": "^7.0.0", - "@babel/generator": "^7.3.4", - "@babel/helpers": "^7.2.0", - "@babel/parser": "^7.3.4", - "@babel/template": "^7.2.2", - "@babel/traverse": "^7.3.4", - "@babel/types": "^7.3.4", + "@babel/generator": "^7.4.0", + "@babel/helpers": "^7.4.3", + "@babel/parser": "^7.4.3", + "@babel/template": "^7.4.0", + "@babel/traverse": "^7.4.3", + "@babel/types": "^7.4.0", "convert-source-map": "^1.1.0", "debug": "^4.1.0", "json5": "^2.1.0", @@ -62,24 +62,18 @@ } }, "@babel/generator": { - "version": "7.3.4", - "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.3.4.tgz", - "integrity": "sha512-8EXhHRFqlVVWXPezBW5keTiQi/rJMQTg/Y9uVCEZ0CAF3PKtCCaVRnp64Ii1ujhkoDhhF1fVsImoN4yJ2uz4Wg==", + "version": "7.4.0", + "resolved": "https://registry.npmjs.org/@babel/generator/-/generator-7.4.0.tgz", + "integrity": "sha512-/v5I+a1jhGSKLgZDcmAUZ4K/VePi43eRkUs3yePW1HB1iANOD5tqJXwGSG4BZhSksP8J9ejSlwGeTiiOFZOrXQ==", "dev": true, "requires": { - "@babel/types": "^7.3.4", + "@babel/types": "^7.4.0", "jsesc": "^2.5.1", "lodash": "^4.17.11", "source-map": "^0.5.0", "trim-right": "^1.0.1" }, "dependencies": { - "jsesc": { - "version": "2.5.2", - "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", - "integrity": "sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==", - "dev": true - }, "source-map": { "version": "0.5.7", "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.5.7.tgz", @@ -115,23 +109,23 @@ "dev": true }, "@babel/helper-split-export-declaration": { - "version": "7.0.0", - "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.0.0.tgz", - "integrity": "sha512-MXkOJqva62dfC0w85mEf/LucPPS/1+04nmmRMPEBUB++hiiThQ2zPtX/mEWQ3mtzCEjIJvPY8nuwxXtQeQwUag==", + "version": "7.4.0", + "resolved": "https://registry.npmjs.org/@babel/helper-split-export-declaration/-/helper-split-export-declaration-7.4.0.tgz", + "integrity": "sha512-7Cuc6JZiYShaZnybDmfwhY4UYHzI6rlqhWjaIqbsJGsIqPimEYy5uh3akSRLMg65LSdSEnJ8a8/bWQN6u2oMGw==", "dev": true, "requires": { - "@babel/types": "^7.0.0" + "@babel/types": "^7.4.0" } }, "@babel/helpers": { - "version": "7.3.1", - "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.3.1.tgz", - "integrity": "sha512-Q82R3jKsVpUV99mgX50gOPCWwco9Ec5Iln/8Vyu4osNIOQgSrd9RFrQeUvmvddFNoLwMyOUWU+5ckioEKpDoGA==", + "version": "7.4.3", + "resolved": "https://registry.npmjs.org/@babel/helpers/-/helpers-7.4.3.tgz", + "integrity": "sha512-BMh7X0oZqb36CfyhvtbSmcWc3GXocfxv3yNsAEuM0l+fAqSO22rQrUpijr3oE/10jCTrB6/0b9kzmG4VetCj8Q==", "dev": true, "requires": { - "@babel/template": "^7.1.2", - "@babel/traverse": "^7.1.5", - "@babel/types": "^7.3.0" + "@babel/template": "^7.4.0", + "@babel/traverse": "^7.4.3", + "@babel/types": "^7.4.0" } }, "@babel/highlight": { @@ -146,9 +140,9 @@ } }, "@babel/parser": { - "version": "7.3.4", - "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.3.4.tgz", - "integrity": "sha512-tXZCqWtlOOP4wgCp6RjRvLmfuhnqTLy9VHwRochJBCP2nDm27JnnuFEnXFASVyQNHk36jD1tAammsCEEqgscIQ==", + "version": "7.4.3", + "resolved": "https://registry.npmjs.org/@babel/parser/-/parser-7.4.3.tgz", + "integrity": "sha512-gxpEUhTS1sGA63EGQGuA+WESPR/6tz6ng7tSHFCmaTJK/cGK8y37cBTspX+U2xCAue2IQVvF6Z0oigmjwD8YGQ==", "dev": true }, "@babel/plugin-syntax-object-rest-spread": { @@ -161,36 +155,36 @@ } }, "@babel/runtime": { - "version": "7.3.4", - "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.3.4.tgz", - "integrity": "sha512-IvfvnMdSaLBateu0jfsYIpZTxAc2cKEXEMiezGGN75QcBcecDUKd3PgLAncT0oOgxKy8dd8hrJKj9MfzgfZd6g==", + "version": "7.4.3", + "resolved": "https://registry.npmjs.org/@babel/runtime/-/runtime-7.4.3.tgz", + "integrity": "sha512-9lsJwJLxDh/T3Q3SZszfWOTkk3pHbkmH+3KY+zwIDmsNlxsumuhS2TH3NIpktU4kNvfzy+k3eLT7aTJSPTo0OA==", "requires": { - "regenerator-runtime": "^0.12.0" + "regenerator-runtime": "^0.13.2" } }, "@babel/template": { - "version": "7.2.2", - "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.2.2.tgz", - "integrity": "sha512-zRL0IMM02AUDwghf5LMSSDEz7sBCO2YnNmpg3uWTZj/v1rcG2BmQUvaGU8GhU8BvfMh1k2KIAYZ7Ji9KXPUg7g==", + "version": "7.4.0", + "resolved": "https://registry.npmjs.org/@babel/template/-/template-7.4.0.tgz", + "integrity": "sha512-SOWwxxClTTh5NdbbYZ0BmaBVzxzTh2tO/TeLTbF6MO6EzVhHTnff8CdBXx3mEtazFBoysmEM6GU/wF+SuSx4Fw==", "dev": true, "requires": { "@babel/code-frame": "^7.0.0", - "@babel/parser": "^7.2.2", - "@babel/types": "^7.2.2" + "@babel/parser": "^7.4.0", + "@babel/types": "^7.4.0" } }, "@babel/traverse": { - "version": "7.3.4", - "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.3.4.tgz", - "integrity": "sha512-TvTHKp6471OYEcE/91uWmhR6PrrYywQntCHSaZ8CM8Vmp+pjAusal4nGB2WCCQd0rvI7nOMKn9GnbcvTUz3/ZQ==", + "version": "7.4.3", + "resolved": "https://registry.npmjs.org/@babel/traverse/-/traverse-7.4.3.tgz", + "integrity": "sha512-HmA01qrtaCwwJWpSKpA948cBvU5BrmviAief/b3AVw936DtcdsTexlbyzNuDnthwhOQ37xshn7hvQaEQk7ISYQ==", "dev": true, "requires": { "@babel/code-frame": "^7.0.0", - "@babel/generator": "^7.3.4", + "@babel/generator": "^7.4.0", "@babel/helper-function-name": "^7.1.0", - "@babel/helper-split-export-declaration": "^7.0.0", - "@babel/parser": "^7.3.4", - "@babel/types": "^7.3.4", + "@babel/helper-split-export-declaration": "^7.4.0", + "@babel/parser": "^7.4.3", + "@babel/types": "^7.4.0", "debug": "^4.1.0", "globals": "^11.1.0", "lodash": "^4.17.11" @@ -208,9 +202,9 @@ } }, "@babel/types": { - "version": "7.3.4", - "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.3.4.tgz", - "integrity": "sha512-WEkp8MsLftM7O/ty580wAmZzN1nDmCACc5+jFzUt+GUFNNIi3LdRlueYz0YIlmJhlZx1QYDMZL5vdWCL0fNjFQ==", + "version": "7.4.0", + "resolved": "https://registry.npmjs.org/@babel/types/-/types-7.4.0.tgz", + "integrity": "sha512-aPvkXyU2SPOnztlgo8n9cEiXW755mgyvueUPcpStqdzoSPm0fjO0vQBjLkt3JKJW7ufikfcnMTTPsN1xaTsBPA==", "dev": true, "requires": { "esutils": "^2.0.2", @@ -441,9 +435,9 @@ "dev": true }, "@types/babel__core": { - "version": "7.1.0", - "resolved": "https://registry.npmjs.org/@types/babel__core/-/babel__core-7.1.0.tgz", - "integrity": "sha512-wJTeJRt7BToFx3USrCDs2BhEi4ijBInTQjOIukj6a/5tEkwpFMVZ+1ppgmE+Q/FQyc5P/VWUbx7I9NELrKruHA==", + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/@types/babel__core/-/babel__core-7.1.1.tgz", + "integrity": "sha512-+hjBtgcFPYyCTo0A15+nxrCVJL7aC6Acg87TXd5OW3QhHswdrOLoles+ldL2Uk8q++7yIfl4tURtztccdeeyOw==", "dev": true, "requires": { "@babel/parser": "^7.1.0", @@ -498,6 +492,29 @@ "resolved": "https://registry.npmjs.org/@types/dom4/-/dom4-2.0.1.tgz", "integrity": "sha512-kSkVAvWmMZiCYtvqjqQEwOmvKwcH+V4uiv3qPQ8pAh1Xl39xggGEo8gHUqV4waYGHezdFw0rKBR8Jt0CrQSDZA==" }, + "@types/events": { + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/@types/events/-/events-3.0.0.tgz", + "integrity": "sha512-EaObqwIvayI5a8dCzhFrjKzVwKLxjoG9T6Ppd5CEo07LRKfQ8Yokw54r5+Wq7FaBQ+yXRvQAYPrHwya1/UFt9g==", + "dev": true + }, + "@types/file-saver": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/@types/file-saver/-/file-saver-2.0.0.tgz", + "integrity": "sha512-dxdRrUov2HVTbSRFX+7xwUPlbGYVEZK6PrSqClg2QPos3PNe0bCajkDDkDeeC1znjSH03KOEqVbXpnJuWa2wgQ==", + "dev": true + }, + "@types/glob": { + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/@types/glob/-/glob-7.1.1.tgz", + "integrity": "sha512-1Bh06cbWJUHMC97acuD6UMG29nMt0Aqz1vF3guLfG+kHHJhy3AyohZFFxYk2f7Q1SQIrNwvncxAE0N/9s70F2w==", + "dev": true, + "requires": { + "@types/events": "*", + "@types/minimatch": "*", + "@types/node": "*" + } + }, "@types/history": { "version": "4.7.2", "resolved": "https://registry.npmjs.org/@types/history/-/history-4.7.2.tgz", @@ -546,16 +563,16 @@ "@types/lodash": "*" } }, - "@types/mocha": { - "version": "5.2.6", - "resolved": "https://registry.npmjs.org/@types/mocha/-/mocha-5.2.6.tgz", - "integrity": "sha512-1axi39YdtBI7z957vdqXI4Ac25e7YihYQtJa+Clnxg1zTJEaIRbndt71O3sP4GAMgiAm0pY26/b9BrY4MR/PMw==", + "@types/minimatch": { + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/@types/minimatch/-/minimatch-3.0.3.tgz", + "integrity": "sha512-tHq6qdbT9U1IRSGf14CL0pUlULksvY9OZ+5eEgl1N7t+OA3tGvNpxJCzuKQlsNgCVwbAs670L1vcVQi8j9HjnA==", "dev": true }, "@types/node": { - "version": "11.13.4", - "resolved": "https://registry.npmjs.org/@types/node/-/node-11.13.4.tgz", - "integrity": "sha512-+rabAZZ3Yn7tF/XPGHupKIL5EcAbrLxnTr/hgQICxbeuAfWtT0UZSfULE+ndusckBItcv4o6ZeOJplQikVcLvQ==", + "version": "11.13.7", + "resolved": "https://registry.npmjs.org/@types/node/-/node-11.13.7.tgz", + "integrity": "sha512-suFHr6hcA9mp8vFrZTgrmqW2ZU3mbWsryQtQlY/QvwTISCw7nw/j+bCQPPohqmskhmqa5wLNuMHTTsc+xf1MQg==", "dev": true }, "@types/numeral": { @@ -565,21 +582,30 @@ "dev": true }, "@types/prop-types": { - "version": "15.7.0", - "resolved": "https://registry.npmjs.org/@types/prop-types/-/prop-types-15.7.0.tgz", - "integrity": "sha512-eItQyV43bj4rR3JPV0Skpl1SncRCdziTEK9/v8VwXmV6d/qOUO8/EuWeHBbCZcsfSHfzI5UyMJLCSXtxxznyZg==", + "version": "15.7.1", + "resolved": "https://registry.npmjs.org/@types/prop-types/-/prop-types-15.7.1.tgz", + "integrity": "sha512-CFzn9idOEpHrgdw8JsoTkaDDyRWk1jrzIV8djzcgpq0y9tG4B4lFT+Nxh52DVpDXV+n4+NPNv7M1Dj5uMp6XFg==", "dev": true }, "@types/react": { - "version": "16.8.13", - "resolved": "https://registry.npmjs.org/@types/react/-/react-16.8.13.tgz", - "integrity": "sha512-otJ4ntMuHGrvm67CdDJMAls4WqotmAmW0g3HmWi9LCjSWXrxoXY/nHXrtmMfvPEEmGFNm6NdgMsJmnfH820Qaw==", + "version": "16.8.14", + "resolved": "https://registry.npmjs.org/@types/react/-/react-16.8.14.tgz", + "integrity": "sha512-26tFVJ1omGmzIdFTFmnC5zhz1GTaqCjxgUxV4KzWvsybF42P7/j4RBn6UeO3KbHPXqKWZszMXMoI65xIWm954A==", "dev": true, "requires": { "@types/prop-types": "*", "csstype": "^2.2.0" } }, + "@types/react-copy-to-clipboard": { + "version": "4.2.6", + "resolved": "https://registry.npmjs.org/@types/react-copy-to-clipboard/-/react-copy-to-clipboard-4.2.6.tgz", + "integrity": "sha512-v4/yLsuPf8GSFuTy9fA1ABpL5uuy04vwW7qs+cfxSe1UU/M/KK95rF3N3GRseismoK9tA28SvpwVsAg/GWoF3A==", + "dev": true, + "requires": { + "@types/react": "*" + } + }, "@types/react-dom": { "version": "16.8.4", "resolved": "https://registry.npmjs.org/@types/react-dom/-/react-dom-16.8.4.tgz", @@ -653,9 +679,9 @@ } }, "@types/yargs": { - "version": "12.0.11", - "resolved": "https://registry.npmjs.org/@types/yargs/-/yargs-12.0.11.tgz", - "integrity": "sha512-IsU1TD+8cQCyG76ZqxP0cVFnofvfzT8p/wO8ENT4jbN/KKN3grsHFgHNl/U+08s33ayX4LwI85cEhYXCOlOkMw==", + "version": "12.0.12", + "resolved": "https://registry.npmjs.org/@types/yargs/-/yargs-12.0.12.tgz", + "integrity": "sha512-SOhuU4wNBxhhTHxYaiG5NY4HBhDIDnJF60GU+2LqHAdKKer86//e4yg69aENCtQ04n0ovz+tq2YPME5t5yp4pw==", "dev": true }, "@webassemblyjs/ast": { @@ -881,9 +907,9 @@ "dev": true }, "acorn-globals": { - "version": "4.3.0", - "resolved": "https://registry.npmjs.org/acorn-globals/-/acorn-globals-4.3.0.tgz", - "integrity": "sha512-hMtHj3s5RnuhvHPowpBYvJVj3rAar82JiDQHvGs1zO0l10ocX/xEdBShNHTJaboucJUsScghp74pH3s7EnHHQw==", + "version": "4.3.2", + "resolved": "https://registry.npmjs.org/acorn-globals/-/acorn-globals-4.3.2.tgz", + "integrity": "sha512-BbzvZhVtZP+Bs1J1HcwrQe8ycfO0wStkSGxuul3He3GkHOIZ6eTqOkPuw9IP1X3+IkOo4wiJmwkobzXYz4wewQ==", "dev": true, "requires": { "acorn": "^6.0.1", @@ -905,9 +931,9 @@ "dev": true }, "ajv": { - "version": "6.9.2", - "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.9.2.tgz", - "integrity": "sha512-4UFy0/LgDo7Oa/+wOAlj44tp9K78u38E5/359eSrqEp1Z5PdVfimCcs7SluXMP755RUQu6d2b4AvF0R1C9RZjg==", + "version": "6.10.0", + "resolved": "https://registry.npmjs.org/ajv/-/ajv-6.10.0.tgz", + "integrity": "sha512-nffhOpkymDECQyR0mnsUtoCE8RlX38G0rYP+wgLWFyZuUyuuojSSvi/+euOiQBIn63whYwYVIIH1TvE3tu4OEg==", "dev": true, "requires": { "fast-deep-equal": "^2.0.1", @@ -935,9 +961,9 @@ "dev": true }, "ansi-colors": { - "version": "3.2.3", - "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-3.2.3.tgz", - "integrity": "sha512-LEHHyuhlPY3TmuUYMh2oz89lTShfvgbmzaBcxve9t/9Wuy7Dwf4yoAKcND7KFT1HAQfqZ12qtc+DUrBMeKF9nw==", + "version": "3.2.4", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-3.2.4.tgz", + "integrity": "sha512-hHUXGagefjN2iRrID63xckIvotOXOojhQKWIPUZ4mNUZ9nLZW+7FMNoE1lOkEhNWYsx/7ysGIuJYCiMAA9FnrA==", "dev": true }, "ansi-escapes": { @@ -975,6 +1001,17 @@ "requires": { "micromatch": "^3.1.4", "normalize-path": "^2.1.1" + }, + "dependencies": { + "normalize-path": { + "version": "2.1.1", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-2.1.1.tgz", + "integrity": "sha1-GrKLVW4Zg2Oowab35vogE3/mrtk=", + "dev": true, + "requires": { + "remove-trailing-separator": "^1.0.1" + } + } } }, "append-transform": { @@ -1048,9 +1085,9 @@ "dev": true }, "array-flatten": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-2.1.2.tgz", - "integrity": "sha512-hNfzcOV8W4NdualtqBFPyVO+54DSJuZGY9qT4pRroB6S9e3iiido2ISIC5h9R2sPJ8H3FHCIiEnsv1lPXO3KtQ==", + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-1.1.1.tgz", + "integrity": "sha1-ml9pkFGx5wczKPKgCJaLZOopVdI=", "dev": true }, "array-includes": { @@ -1168,9 +1205,9 @@ } }, "async-each": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/async-each/-/async-each-1.0.1.tgz", - "integrity": "sha1-GdOGodntxufByF04iu28xW0zYC0=", + "version": "1.0.3", + "resolved": "https://registry.npmjs.org/async-each/-/async-each-1.0.3.tgz", + "integrity": "sha512-z/WhQ5FPySLdvREByI2vZiTWwCnF0moMJ1hK9YQwDTHKh6I7/uSckMetoRGb5UBZPC1z0jlw+n/XCgjeH7y1AQ==", "dev": true }, "async-foreach": { @@ -1243,65 +1280,6 @@ "is-buffer": "^1.1.5" } }, - "babel-code-frame": { - "version": "6.26.0", - "resolved": "https://registry.npmjs.org/babel-code-frame/-/babel-code-frame-6.26.0.tgz", - "integrity": "sha1-Y/1D99weO7fONZR9uP42mj9Yx0s=", - "dev": true, - "requires": { - "chalk": "^1.1.3", - "esutils": "^2.0.2", - "js-tokens": "^3.0.2" - }, - "dependencies": { - "ansi-regex": { - "version": "2.1.1", - "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-2.1.1.tgz", - "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8=", - "dev": true - }, - "ansi-styles": { - "version": "2.2.1", - "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-2.2.1.tgz", - "integrity": "sha1-tDLdM1i2NM914eRmQ2gkBTPB3b4=", - "dev": true - }, - "chalk": { - "version": "1.1.3", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-1.1.3.tgz", - "integrity": "sha1-qBFcVeSnAv5NFQq9OHKCKn4J/Jg=", - "dev": true, - "requires": { - "ansi-styles": "^2.2.1", - "escape-string-regexp": "^1.0.2", - "has-ansi": "^2.0.0", - "strip-ansi": "^3.0.0", - "supports-color": "^2.0.0" - } - }, - "js-tokens": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/js-tokens/-/js-tokens-3.0.2.tgz", - "integrity": "sha1-mGbfOVECEw449/mWvOtlRDIJwls=", - "dev": true - }, - "strip-ansi": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-3.0.1.tgz", - "integrity": "sha1-ajhfuIU9lS1f8F0Oiq+UJ43GPc8=", - "dev": true, - "requires": { - "ansi-regex": "^2.0.0" - } - }, - "supports-color": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-2.0.0.tgz", - "integrity": "sha1-U10EXOa2Nj+kARcIRimZXp3zJMc=", - "dev": true - } - } - }, "babel-jest": { "version": "24.7.1", "resolved": "https://registry.npmjs.org/babel-jest/-/babel-jest-24.7.1.tgz", @@ -1318,14 +1296,14 @@ } }, "babel-plugin-istanbul": { - "version": "5.1.1", - "resolved": "https://registry.npmjs.org/babel-plugin-istanbul/-/babel-plugin-istanbul-5.1.1.tgz", - "integrity": "sha512-RNNVv2lsHAXJQsEJ5jonQwrJVWK8AcZpG1oxhnjCUaAjL7xahYLANhPUZbzEQHjKy1NMYUwn+0NPKQc8iSY4xQ==", + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/babel-plugin-istanbul/-/babel-plugin-istanbul-5.1.2.tgz", + "integrity": "sha512-U3ZVajC+Z69Gim7ZzmD4Wcsq76i/1hqDamBfowc1tWzWjybRy70iWfngP2ME+1CrgcgZ/+muIbPY/Yi0dxdIkQ==", "dev": true, "requires": { "find-up": "^3.0.0", - "istanbul-lib-instrument": "^3.0.0", - "test-exclude": "^5.0.0" + "istanbul-lib-instrument": "^3.2.0", + "test-exclude": "^5.2.2" } }, "babel-plugin-jest-hoist": { @@ -1435,6 +1413,18 @@ "tweetnacl": "^0.14.3" } }, + "bfj": { + "version": "6.1.1", + "resolved": "https://registry.npmjs.org/bfj/-/bfj-6.1.1.tgz", + "integrity": "sha512-+GUNvzHR4nRyGybQc2WpNJL4MJazMuvf92ueIyA0bIkPRwhhQu3IfZQ2PSoVPpCBJfmoSdOxu5rnotfFLlvYRQ==", + "dev": true, + "requires": { + "bluebird": "^3.5.1", + "check-types": "^7.3.0", + "hoopy": "^0.1.2", + "tryer": "^1.0.0" + } + }, "big.js": { "version": "5.2.2", "resolved": "https://registry.npmjs.org/big.js/-/big.js-5.2.2.tgz", @@ -1442,9 +1432,9 @@ "dev": true }, "binary-extensions": { - "version": "1.13.0", - "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-1.13.0.tgz", - "integrity": "sha512-EgmjVLMn22z7eGGv3kcnHwSnJXmFHjISTY9E/S5lIcTD3Oxw05QTcBLNkJFzcb3cNueUdF/IN4U+d78V0zO8Hw==", + "version": "1.13.1", + "resolved": "https://registry.npmjs.org/binary-extensions/-/binary-extensions-1.13.1.tgz", + "integrity": "sha512-Un7MIEDdUC5gNpcGDV97op1Ywk748MpHcFTHoYs6qnj1Z3j7I53VG3nwZhKzoBZmbdRNnb6WRdFlwl7tSDuZGw==", "dev": true }, "block-stream": { @@ -1524,6 +1514,14 @@ "dns-txt": "^2.0.2", "multicast-dns": "^6.0.1", "multicast-dns-service-types": "^1.1.0" + }, + "dependencies": { + "array-flatten": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-2.1.2.tgz", + "integrity": "sha512-hNfzcOV8W4NdualtqBFPyVO+54DSJuZGY9qT4pRroB6S9e3iiido2ISIC5h9R2sPJ8H3FHCIiEnsv1lPXO3KtQ==", + "dev": true + } } }, "brace": { @@ -1599,12 +1597,6 @@ } } }, - "browser-stdout": { - "version": "1.3.1", - "resolved": "https://registry.npmjs.org/browser-stdout/-/browser-stdout-1.3.1.tgz", - "integrity": "sha512-qhAVI1+Av2X7qelOfAIYwXONood6XlZE/fXaBSmW/T5SzLAmCgzi+eiWE7fUvbHaeNBQH13UftjpXxsfLkMpgw==", - "dev": true - }, "browserify-aes": { "version": "1.2.0", "resolved": "https://registry.npmjs.org/browserify-aes/-/browserify-aes-1.2.0.tgz", @@ -1677,14 +1669,14 @@ } }, "browserslist": { - "version": "4.5.4", - "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.5.4.tgz", - "integrity": "sha512-rAjx494LMjqKnMPhFkuLmLp8JWEX0o8ADTGeAbOqaF+XCvYLreZrG5uVjnPBlAQ8REZK4pzXGvp0bWgrFtKaag==", + "version": "4.5.5", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.5.5.tgz", + "integrity": "sha512-0QFO1r/2c792Ohkit5XI8Cm8pDtZxgNl2H6HU4mHrpYz7314pEYcsAVVatM0l/YmxPnEzh9VygXouj4gkFUTKA==", "dev": true, "requires": { - "caniuse-lite": "^1.0.30000955", - "electron-to-chromium": "^1.3.122", - "node-releases": "^1.1.13" + "caniuse-lite": "^1.0.30000960", + "electron-to-chromium": "^1.3.124", + "node-releases": "^1.1.14" } }, "bs-logger": { @@ -1849,15 +1841,15 @@ } }, "callsites": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/callsites/-/callsites-3.0.0.tgz", - "integrity": "sha512-tWnkwu9YEq2uzlBDI4RcLn8jrFvF9AOi8PxDNU3hZZjJcjkcRAq3vCI+vZcg1SuxISDYe86k9VZFwAxDiJGoAw==", + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/callsites/-/callsites-3.1.0.tgz", + "integrity": "sha512-P8BjAsXvZS+VIDUI11hHCQEv74YT67YUi5JJFNWIqL235sBmjX4+qx9Muvls5ivyNENctx46xQLQ3aTuE7ssaQ==", "dev": true }, "camelcase": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-5.0.0.tgz", - "integrity": "sha512-faqwZqnWxbxn+F1d399ygeamQNy3lPp/H9H6rNrqYh4FSVCtcY+3cub1MxA8o9mDd55mM8Aghuu/kuyYA6VTsA==", + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-5.3.1.tgz", + "integrity": "sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg==", "dev": true }, "camelcase-keys": { @@ -1879,9 +1871,9 @@ } }, "caniuse-lite": { - "version": "1.0.30000957", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30000957.tgz", - "integrity": "sha512-8wxNrjAzyiHcLXN/iunskqQnJquQQ6VX8JHfW5kLgAPRSiSuKZiNfmIkP5j7jgyXqAQBSoXyJxfnbCFS0ThSiQ==", + "version": "1.0.30000962", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30000962.tgz", + "integrity": "sha512-WXYsW38HK+6eaj5IZR16Rn91TGhU3OhbwjKZvJ4HN/XBIABLKfbij9Mnd3pM0VEwZSlltWjoWg3I8FQ0DGgNOA==", "dev": true }, "capture-exit": { @@ -1951,10 +1943,16 @@ "integrity": "sha512-7I/xceXfKyUJmSAn/jw8ve/9DyOP7XxufNYLI9Px7CmsKgEUaZLUTax6nZxGQtaoiZCjpu6cHPj20xC/vqRReQ==", "dev": true }, + "check-types": { + "version": "7.4.0", + "resolved": "https://registry.npmjs.org/check-types/-/check-types-7.4.0.tgz", + "integrity": "sha512-YbulWHdfP99UfZ73NcUDlNJhEIDgm9Doq9GhpyXbF+7Aegi3CVV7qqMCKTTqJxlvEvnQBp9IA+dxsGN6xK/nSg==", + "dev": true + }, "chokidar": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-2.1.2.tgz", - "integrity": "sha512-IwXUx0FXc5ibYmPC2XeEj5mpXoV66sR+t3jqu2NS2GYwCktt3KF1/Qqjws/NkegajBA4RbZ5+DDwlOiJsxDHEg==", + "version": "2.1.5", + "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-2.1.5.tgz", + "integrity": "sha512-i0TprVWp+Kj4WRPtInjexJ8Q+BqTE909VpH8xVhXrJkoc5QC8VO9TryGOqTr+2hljzc1sC62t22h5tZePodM/A==", "dev": true, "requires": { "anymatch": "^2.0.0", @@ -1968,15 +1966,7 @@ "normalize-path": "^3.0.0", "path-is-absolute": "^1.0.0", "readdirp": "^2.2.1", - "upath": "^1.1.0" - }, - "dependencies": { - "normalize-path": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", - "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", - "dev": true - } + "upath": "^1.1.1" } }, "chownr": { @@ -2141,9 +2131,9 @@ } }, "commander": { - "version": "2.19.0", - "resolved": "https://registry.npmjs.org/commander/-/commander-2.19.0.tgz", - "integrity": "sha512-6tvAOO+D6OENvRAh524Dh9jcfKTYDQAqvqezbCW82xj5X0pSrcpxtvRKHLG0yBY6SD7PSDrJaj+0AiOcKVd1Xg==", + "version": "2.20.0", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.20.0.tgz", + "integrity": "sha512-7j2y+40w61zy6YC2iRNpUe/NwhNyoXrYpHMrSunaMG64nRnaf96zO/KMQR4OyN/UnE5KLyEBnKHd4aG3rskjpQ==", "dev": true }, "commondir": { @@ -2159,9 +2149,9 @@ "dev": true }, "component-emitter": { - "version": "1.2.1", - "resolved": "https://registry.npmjs.org/component-emitter/-/component-emitter-1.2.1.tgz", - "integrity": "sha1-E3kY1teCg/ffemt8WmPhQOaUJeY=", + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/component-emitter/-/component-emitter-1.3.0.tgz", + "integrity": "sha512-Rd3se6QB+sO1TwqZjscQrurpEPIfO0/yYnSin6Q/rD3mOutHvUrCAhJub3r90uNb+SESBuE0QYoB90YdfatsRg==", "dev": true }, "compressible": { @@ -2303,6 +2293,14 @@ "integrity": "sha1-Z29us8OZl8LuGsOpJP1hJHSPV40=", "dev": true }, + "copy-to-clipboard": { + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/copy-to-clipboard/-/copy-to-clipboard-3.2.0.tgz", + "integrity": "sha512-eOZERzvCmxS8HWzugj4Uxl8OJxa7T2k1Gi0X5qavwydHIfuSHq2dTD09LOg/XyGq4Zpb5IsR/2OJ5lbOegz78w==", + "requires": { + "toggle-selection": "^1.0.6" + } + }, "core-js": { "version": "1.2.7", "resolved": "https://registry.npmjs.org/core-js/-/core-js-1.2.7.tgz", @@ -2421,6 +2419,25 @@ "requires": { "postcss": "^7.0.6", "postcss-selector-parser": "^5.0.0-rc.4" + }, + "dependencies": { + "cssesc": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-2.0.0.tgz", + "integrity": "sha512-MsCAG1z9lPdoO/IUMLSBWBSVxVtJ1395VGIQ+Fc2gNdkQ1hNDnQdw3YhA71WJCBW1vdwA0cAnk/DnW6bqoEUYg==", + "dev": true + }, + "postcss-selector-parser": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-5.0.0.tgz", + "integrity": "sha512-w+zLE5Jhg6Liz8+rQOWEAwtwkyqpfnmsinXjXg6cY7YIONZZtgvE0v2O0uhQBs0peNomOJwWRKt6JBfTdTd3OQ==", + "dev": true, + "requires": { + "cssesc": "^2.0.0", + "indexes-of": "^1.0.1", + "uniq": "^1.0.1" + } + } } }, "css-loader": { @@ -2440,20 +2457,6 @@ "postcss-modules-values": "^2.0.0", "postcss-value-parser": "^3.3.0", "schema-utils": "^1.0.0" - }, - "dependencies": { - "camelcase": { - "version": "5.3.1", - "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-5.3.1.tgz", - "integrity": "sha512-L28STB170nwWS63UjtlEOE3dldQApaJXZkOI1uMFfzf3rRuPegHaHesyee+YxQ+W6SvRDQV6UrdOdRiR153wJg==", - "dev": true - }, - "normalize-path": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", - "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", - "dev": true - } } }, "css-parse": { @@ -2499,9 +2502,9 @@ } }, "csstype": { - "version": "2.6.3", - "resolved": "https://registry.npmjs.org/csstype/-/csstype-2.6.3.tgz", - "integrity": "sha512-rINUZXOkcBmoHWEyu7JdHu5JMzkGRoMX4ov9830WNgxf5UYxcBUO0QTKAqeJ5EZfSdlrcJYkC8WwfVW7JYi4yg==", + "version": "2.6.4", + "resolved": "https://registry.npmjs.org/csstype/-/csstype-2.6.4.tgz", + "integrity": "sha512-lAJUJP3M6HxFXbqtGRc0iZrdyeN+WzOWeY0q/VnFzI+kqVrYIzC7bWlKqCW7oCIdzoPkvfp82EVvrTlQ8zsWQg==", "dev": true }, "currently-unhandled": { @@ -2929,6 +2932,12 @@ "resolved": "https://registry.npmjs.org/druid-console/-/druid-console-0.0.2.tgz", "integrity": "sha512-0sYnfUHHMoajaud/i5BHKA12bUxiWEHJ9rxGqVEppFxsEcxef0TZQ5J59lU+UniEBcz/sG5fTESRyS7cOm3tSQ==" }, + "duplexer": { + "version": "0.1.1", + "resolved": "https://registry.npmjs.org/duplexer/-/duplexer-0.1.1.tgz", + "integrity": "sha1-rOb/gIwc5mtX0ev5eXessCM0z8E=", + "dev": true + }, "duplexify": { "version": "3.7.1", "resolved": "https://registry.npmjs.org/duplexify/-/duplexify-3.7.1.tgz", @@ -2957,10 +2966,16 @@ "integrity": "sha1-WQxhFWsK4vTwJVcyoViyZrxWsh0=", "dev": true }, + "ejs": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/ejs/-/ejs-2.6.1.tgz", + "integrity": "sha512-0xy4A/twfrRCnkhfk8ErDi5DqdAsAqeGxht4xkCUrsvhhbQNs7E+4jV0CN7+NKIY0aHE72+XvqtBIXzD31ZbXQ==", + "dev": true + }, "electron-to-chromium": { - "version": "1.3.124", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.3.124.tgz", - "integrity": "sha512-glecGr/kFdfeXUHOHAWvGcXrxNU+1wSO/t5B23tT1dtlvYB26GY8aHzZSWD7HqhqC800Lr+w/hQul6C5AF542w==", + "version": "1.3.125", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.3.125.tgz", + "integrity": "sha512-XxowpqQxJ4nDwUXHtVtmEhRqBpm2OnjBomZmZtHD0d2Eo0244+Ojezhk3sD/MBSSe2nxCdGQFRXHIsf/LUTL9A==", "dev": true }, "elliptic": { @@ -3346,12 +3361,6 @@ "vary": "~1.1.2" }, "dependencies": { - "array-flatten": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/array-flatten/-/array-flatten-1.1.1.tgz", - "integrity": "sha1-ml9pkFGx5wczKPKgCJaLZOopVdI=", - "dev": true - }, "debug": { "version": "2.6.9", "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", @@ -3552,6 +3561,11 @@ "flat-cache": "^2.0.1" } }, + "file-saver": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/file-saver/-/file-saver-2.0.1.tgz", + "integrity": "sha512-dCB3K7/BvAcUmtmh1DzFdv0eXSVJ9IAFt1mw3XZfAexodNRoE29l3xB2EX4wH2q8m/UTzwzEPq/ArYk98kUkBQ==" + }, "fileset": { "version": "2.0.3", "resolved": "https://registry.npmjs.org/fileset/-/fileset-2.0.3.tgz", @@ -3562,6 +3576,12 @@ "minimatch": "^3.0.3" } }, + "filesize": { + "version": "3.6.1", + "resolved": "https://registry.npmjs.org/filesize/-/filesize-3.6.1.tgz", + "integrity": "sha512-7KjR1vv6qnicaPMi1iiTcI85CyYwRO/PSFCu6SvqL8jN2Wjt/NIYQTFtFs7fSDCYOstUkEWIQGFUg5YZQfjlcg==", + "dev": true + }, "fill-range": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-4.0.0.tgz", @@ -3626,24 +3646,6 @@ "commondir": "^1.0.1", "make-dir": "^2.0.0", "pkg-dir": "^3.0.0" - }, - "dependencies": { - "make-dir": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-2.1.0.tgz", - "integrity": "sha512-LS9X+dc8KLxXCb8dni79fLIIUA5VyZoyjSMCwTluaXA0o27cCK0bhXkpgw+sTXVpPy/lSO57ilRixqk0vDmtRA==", - "dev": true, - "requires": { - "pify": "^4.0.1", - "semver": "^5.6.0" - } - }, - "pify": { - "version": "4.0.1", - "resolved": "https://registry.npmjs.org/pify/-/pify-4.0.1.tgz", - "integrity": "sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g==", - "dev": true - } } }, "find-up": { @@ -3678,23 +3680,6 @@ } } }, - "flat": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/flat/-/flat-4.1.0.tgz", - "integrity": "sha512-Px/TiLIznH7gEDlPXcUD4KnBusa6kR6ayRUVcnEAbreRIuhkqow/mun59BuRXwoYk7ZQOLW1ZM05ilIvK38hFw==", - "dev": true, - "requires": { - "is-buffer": "~2.0.3" - }, - "dependencies": { - "is-buffer": { - "version": "2.0.3", - "resolved": "https://registry.npmjs.org/is-buffer/-/is-buffer-2.0.3.tgz", - "integrity": "sha512-U15Q7MXTuZlrbymiz95PJpZxu8IlipAp4dtS3wOdgPXx3mqBnslrWU14kxfHB+Py/+2PVKSr37dMAgM2A4uArw==", - "dev": true - } - } - }, "flat-cache": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/flat-cache/-/flat-cache-2.0.1.tgz", @@ -3829,14 +3814,14 @@ "dev": true }, "fsevents": { - "version": "1.2.7", - "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-1.2.7.tgz", - "integrity": "sha512-Pxm6sI2MeBD7RdD12RYsqaP0nMiwx8eZBXCa6z2L+mRHm2DYrOYwihmhjpkdjUHwQhslWQjRpEgNq4XvBmaAuw==", + "version": "1.2.8", + "resolved": "https://registry.npmjs.org/fsevents/-/fsevents-1.2.8.tgz", + "integrity": "sha512-tPvHgPGB7m40CZ68xqFGkKuzN+RnpGmSV+hgeKxhRpbxdqKXUFJGC3yonBOLzQBcJyGpdZFDfCsdOC2KFsXzeA==", "dev": true, "optional": true, "requires": { - "nan": "^2.9.2", - "node-pre-gyp": "^0.10.0" + "nan": "^2.12.1", + "node-pre-gyp": "^0.12.0" }, "dependencies": { "abbrev": { @@ -3848,7 +3833,8 @@ "ansi-regex": { "version": "2.1.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "aproba": { "version": "1.2.0", @@ -3869,12 +3855,14 @@ "balanced-match": { "version": "1.0.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "brace-expansion": { "version": "1.1.11", "bundled": true, "dev": true, + "optional": true, "requires": { "balanced-match": "^1.0.0", "concat-map": "0.0.1" @@ -3889,17 +3877,20 @@ "code-point-at": { "version": "1.1.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "concat-map": { "version": "0.0.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "console-control-strings": { "version": "1.1.0", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "core-util-is": { "version": "1.0.2", @@ -3908,12 +3899,12 @@ "optional": true }, "debug": { - "version": "2.6.9", + "version": "4.1.1", "bundled": true, "dev": true, "optional": true, "requires": { - "ms": "2.0.0" + "ms": "^2.1.1" } }, "deep-extend": { @@ -4016,7 +4007,8 @@ "inherits": { "version": "2.0.3", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "ini": { "version": "1.3.5", @@ -4028,6 +4020,7 @@ "version": "1.0.0", "bundled": true, "dev": true, + "optional": true, "requires": { "number-is-nan": "^1.0.0" } @@ -4042,6 +4035,7 @@ "version": "3.0.4", "bundled": true, "dev": true, + "optional": true, "requires": { "brace-expansion": "^1.1.7" } @@ -4049,12 +4043,14 @@ "minimist": { "version": "0.0.8", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "minipass": { "version": "2.3.5", "bundled": true, "dev": true, + "optional": true, "requires": { "safe-buffer": "^5.1.2", "yallist": "^3.0.0" @@ -4073,29 +4069,30 @@ "version": "0.5.1", "bundled": true, "dev": true, + "optional": true, "requires": { "minimist": "0.0.8" } }, "ms": { - "version": "2.0.0", + "version": "2.1.1", "bundled": true, "dev": true, "optional": true }, "needle": { - "version": "2.2.4", + "version": "2.3.0", "bundled": true, "dev": true, "optional": true, "requires": { - "debug": "^2.1.2", + "debug": "^4.1.0", "iconv-lite": "^0.4.4", "sax": "^1.2.4" } }, "node-pre-gyp": { - "version": "0.10.3", + "version": "0.12.0", "bundled": true, "dev": true, "optional": true, @@ -4123,13 +4120,13 @@ } }, "npm-bundled": { - "version": "1.0.5", + "version": "1.0.6", "bundled": true, "dev": true, "optional": true }, "npm-packlist": { - "version": "1.2.0", + "version": "1.4.1", "bundled": true, "dev": true, "optional": true, @@ -4153,7 +4150,8 @@ "number-is-nan": { "version": "1.0.1", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "object-assign": { "version": "4.1.1", @@ -4165,6 +4163,7 @@ "version": "1.4.0", "bundled": true, "dev": true, + "optional": true, "requires": { "wrappy": "1" } @@ -4250,7 +4249,8 @@ "safe-buffer": { "version": "5.1.2", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "safer-buffer": { "version": "2.1.2", @@ -4265,7 +4265,7 @@ "optional": true }, "semver": { - "version": "5.6.0", + "version": "5.7.0", "bundled": true, "dev": true, "optional": true @@ -4286,6 +4286,7 @@ "version": "1.0.2", "bundled": true, "dev": true, + "optional": true, "requires": { "code-point-at": "^1.0.0", "is-fullwidth-code-point": "^1.0.0", @@ -4305,6 +4306,7 @@ "version": "3.0.1", "bundled": true, "dev": true, + "optional": true, "requires": { "ansi-regex": "^2.0.0" } @@ -4348,12 +4350,14 @@ "wrappy": { "version": "1.0.2", "bundled": true, - "dev": true + "dev": true, + "optional": true }, "yallist": { "version": "3.0.3", "bundled": true, - "dev": true + "dev": true, + "optional": true } } }, @@ -4514,27 +4518,23 @@ "dev": true }, "global-modules": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/global-modules/-/global-modules-1.0.0.tgz", - "integrity": "sha512-sKzpEkf11GpOFuw0Zzjzmt4B4UZwjOcG757PPvrfhxcLFbq0wpsgpOqxpxtxFiCG4DtG93M6XRVbF2oGdev7bg==", + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/global-modules/-/global-modules-2.0.0.tgz", + "integrity": "sha512-NGbfmJBp9x8IxyJSd1P+otYK8vonoJactOogrVfFRIAEY1ukil8RSKDz2Yo7wh1oihl51l/r6W4epkeKJHqL8A==", "dev": true, "requires": { - "global-prefix": "^1.0.1", - "is-windows": "^1.0.1", - "resolve-dir": "^1.0.0" + "global-prefix": "^3.0.0" } }, "global-prefix": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/global-prefix/-/global-prefix-1.0.2.tgz", - "integrity": "sha1-2/dDxsFJklk8ZVVoy2btMsASLr4=", + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/global-prefix/-/global-prefix-3.0.0.tgz", + "integrity": "sha512-awConJSVCHVGND6x3tmMaKcQvwXLhjdkmomy2W+Goaui8YPgYgXJZewhg3fWC+DlfqqQuWg8AwqjGTD2nAPVWg==", "dev": true, "requires": { - "expand-tilde": "^2.0.2", - "homedir-polyfill": "^1.0.1", - "ini": "^1.3.4", - "is-windows": "^1.0.1", - "which": "^1.2.14" + "ini": "^1.3.5", + "kind-of": "^6.0.2", + "which": "^1.3.1" } }, "globals": { @@ -4544,13 +4544,14 @@ "dev": true }, "globby": { - "version": "9.0.0", - "resolved": "https://registry.npmjs.org/globby/-/globby-9.0.0.tgz", - "integrity": "sha512-q0qiO/p1w/yJ0hk8V9x1UXlgsXUxlGd0AHUOXZVXBO6aznDtpx7M8D1kBrCAItoPm+4l8r6ATXV1JpjY2SBQOw==", + "version": "9.2.0", + "resolved": "https://registry.npmjs.org/globby/-/globby-9.2.0.tgz", + "integrity": "sha512-ollPHROa5mcxDEkwg6bPt3QbEf4pDQSNtd6JPL1YvOvAo/7/0VAm9TccUeoTmarjPw4pfUthSCqcyfNB1I3ZSg==", "dev": true, "requires": { + "@types/glob": "^7.1.1", "array-union": "^1.0.2", - "dir-glob": "^2.2.1", + "dir-glob": "^2.2.2", "fast-glob": "^2.2.6", "glob": "^7.1.3", "ignore": "^4.0.3", @@ -4606,12 +4607,6 @@ "integrity": "sha512-6uHUhOPEBgQ24HM+r6b/QwWfZq+yiFcipKFrOFiBEnWdy5sdzYoi+pJeQaPI5qOLRFqWmAXUPQNsielzdLoecA==", "dev": true }, - "growl": { - "version": "1.10.5", - "resolved": "https://registry.npmjs.org/growl/-/growl-1.10.5.tgz", - "integrity": "sha512-qBr4OuELkhPenW6goKVXiv47US3clb3/IbuWF9KNKEijAy9oeHxU9IgzjvJhHkUzhaj7rOUD7+YGWqUjLp5oSA==", - "dev": true - }, "growly": { "version": "1.3.0", "resolved": "https://registry.npmjs.org/growly/-/growly-1.3.0.tgz", @@ -4623,6 +4618,24 @@ "resolved": "https://registry.npmjs.org/gud/-/gud-1.0.0.tgz", "integrity": "sha512-zGEOVKFM5sVPPrYs7J5/hYEw2Pof8KCyOwyhG8sAF26mCAeUFAcYPu1mwB7hhpIP29zOIBaDqwuHdLp0jvZXjw==" }, + "gzip-size": { + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/gzip-size/-/gzip-size-5.1.0.tgz", + "integrity": "sha512-wfSnvypBDRW94v5W3ckvvz/zFUNdJ81VgOP6tE4bPpRUcc0wGqU+y0eZjJEvKxwubJFix6P84sE8M51YWLT7rQ==", + "dev": true, + "requires": { + "duplexer": "^0.1.1", + "pify": "^4.0.1" + }, + "dependencies": { + "pify": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/pify/-/pify-4.0.1.tgz", + "integrity": "sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g==", + "dev": true + } + } + }, "handle-thing": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/handle-thing/-/handle-thing-2.0.0.tgz", @@ -4630,9 +4643,9 @@ "dev": true }, "handlebars": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/handlebars/-/handlebars-4.1.1.tgz", - "integrity": "sha512-3Zhi6C0euYZL5sM0Zcy7lInLXKQ+YLcF/olbN010mzGQ4XVm50JeyBnMqofHh696GrciGruC7kCcApPDJvVgwA==", + "version": "4.1.2", + "resolved": "https://registry.npmjs.org/handlebars/-/handlebars-4.1.2.tgz", + "integrity": "sha512-nvfrjqvt9xQ8Z/w0ijewdD/vvWDTOweBUm96NTr66Wfvo1mJenBLwcYmPs3TIBP5ruzYGD7Hx/DaM9RmhroGPw==", "dev": true, "requires": { "neo-async": "^2.6.0", @@ -4757,12 +4770,6 @@ "minimalistic-assert": "^1.0.1" } }, - "he": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/he/-/he-1.2.0.tgz", - "integrity": "sha512-F/1DnUGPopORZi0ni+CvrCgHQ5FyEAHRLSApuYWMmrbSwoN2Mn/7k+Gl38gJnR7yyDZk6WLXwiGod1JOWNDKGw==", - "dev": true - }, "history": { "version": "4.9.0", "resolved": "https://registry.npmjs.org/history/-/history-4.9.0.tgz", @@ -4809,6 +4816,12 @@ "parse-passwd": "^1.0.0" } }, + "hoopy": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/hoopy/-/hoopy-0.1.4.tgz", + "integrity": "sha512-HRcs+2mr52W0K+x8RzcLzuPPmVIKMSv97RGHy0Ea9y/mpcaK+xTrjICA04KAHi4GRzxliNqNJEFYWHghy3rSfQ==", + "dev": true + }, "hosted-git-info": { "version": "2.7.1", "resolved": "https://registry.npmjs.org/hosted-git-info/-/hosted-git-info-2.7.1.tgz", @@ -5102,14 +5115,6 @@ "requires": { "default-gateway": "^4.2.0", "ipaddr.js": "^1.9.0" - }, - "dependencies": { - "ipaddr.js": { - "version": "1.9.0", - "resolved": "https://registry.npmjs.org/ipaddr.js/-/ipaddr.js-1.9.0.tgz", - "integrity": "sha512-M4Sjn6N/+O6/IXSJseKqHoFc+5FdGJ22sXqnjTpdZweHK64MzEPAyQZyEU3R/KRv2GLoa7nNtg/C2Ev6m7z+eA==", - "dev": true - } } }, "interpret": { @@ -5146,9 +5151,9 @@ "dev": true }, "ipaddr.js": { - "version": "1.8.0", - "resolved": "https://registry.npmjs.org/ipaddr.js/-/ipaddr.js-1.8.0.tgz", - "integrity": "sha1-6qM9bd16zo9/b+DJygRA5wZzix4=", + "version": "1.9.0", + "resolved": "https://registry.npmjs.org/ipaddr.js/-/ipaddr.js-1.9.0.tgz", + "integrity": "sha512-M4Sjn6N/+O6/IXSJseKqHoFc+5FdGJ22sXqnjTpdZweHK64MzEPAyQZyEU3R/KRv2GLoa7nNtg/C2Ev6m7z+eA==", "dev": true }, "is-accessor-descriptor": { @@ -5311,15 +5316,15 @@ "dev": true }, "is-generator-fn": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/is-generator-fn/-/is-generator-fn-2.0.0.tgz", - "integrity": "sha512-elzyIdM7iKoFHzcrndIqjYomImhxrFRnGP3galODoII4TB9gI7mZ+FnlLQmmjf27SxHS2gKEeyhX5/+YRS6H9g==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-generator-fn/-/is-generator-fn-2.1.0.tgz", + "integrity": "sha512-cTIB4yPYL/Grw0EaSzASzg6bBy9gqCofvWN8okThAYIxKJZC+udlRAmGbM0XLeniEJSs8uEgHPGuHSe1XsOLSQ==", "dev": true }, "is-glob": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.0.tgz", - "integrity": "sha1-lSHHaEXMJhCoUgPd8ICpWML/q8A=", + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-4.0.1.tgz", + "integrity": "sha512-5G0tKtBTFImOqDnLB2hG6Bp2qcKEFduo4tZu9MT/H6NQv/ghhy30o55ufafxJ/LdH79LLs2Kfrn85TLKyA7BUg==", "dev": true, "requires": { "is-extglob": "^2.1.1" @@ -5358,27 +5363,27 @@ "dev": true }, "is-path-cwd": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/is-path-cwd/-/is-path-cwd-2.0.0.tgz", - "integrity": "sha512-m5dHHzpOXEiv18JEORttBO64UgTEypx99vCxQLjbBvGhOJxnTNglYoFXxwo6AbsQb79sqqycQEHv2hWkHZAijA==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-path-cwd/-/is-path-cwd-2.1.0.tgz", + "integrity": "sha512-Sc5j3/YnM8tDeyCsVeKlm/0p95075DyLmDEIkSgQ7mXkrOX+uTCtmQFm0CYzVyJwcCCmO3k8qfJt17SxQwB5Zw==", "dev": true }, "is-path-in-cwd": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/is-path-in-cwd/-/is-path-in-cwd-2.0.0.tgz", - "integrity": "sha512-6Vz5Gc9s/sDA3JBVu0FzWufm8xaBsqy1zn8Q6gmvGP6nSDMw78aS4poBNeatWjaRpTpxxLn1WOndAiOlk+qY8A==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-path-in-cwd/-/is-path-in-cwd-2.1.0.tgz", + "integrity": "sha512-rNocXHgipO+rvnP6dk3zI20RpOtrAM/kzbB258Uw5BWr3TpXi861yzjo16Dn4hUox07iw5AyeMLHWsujkjzvRQ==", "dev": true, "requires": { - "is-path-inside": "^1.0.0" + "is-path-inside": "^2.1.0" } }, "is-path-inside": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/is-path-inside/-/is-path-inside-1.0.1.tgz", - "integrity": "sha1-jvW33lBDej/cprToZe96pVy0gDY=", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/is-path-inside/-/is-path-inside-2.1.0.tgz", + "integrity": "sha512-wiyhTzfDWsvwAW53OBWF5zuvaOGlZ6PwYxAbPVDhpm+gM09xKQGjBq/8uYN12aDvMxnAnq3dxTyoSoRNmg5YFg==", "dev": true, "requires": { - "path-is-inside": "^1.0.1" + "path-is-inside": "^1.0.2" } }, "is-plain-obj": { @@ -5498,45 +5503,45 @@ "dev": true }, "istanbul-api": { - "version": "2.1.1", - "resolved": "https://registry.npmjs.org/istanbul-api/-/istanbul-api-2.1.1.tgz", - "integrity": "sha512-kVmYrehiwyeBAk/wE71tW6emzLiHGjYIiDrc8sfyty4F8M02/lrgXSm+R1kXysmF20zArvmZXjlE/mg24TVPJw==", + "version": "2.1.5", + "resolved": "https://registry.npmjs.org/istanbul-api/-/istanbul-api-2.1.5.tgz", + "integrity": "sha512-meYk1BwDp59Pfse1TvPrkKYgVqAufbdBLEVoqvu/hLLKSaQ054ZTksbNepyc223tMnWdm6AdK2URIJJRqdP87g==", "dev": true, "requires": { "async": "^2.6.1", "compare-versions": "^3.2.1", "fileset": "^2.0.3", - "istanbul-lib-coverage": "^2.0.3", - "istanbul-lib-hook": "^2.0.3", - "istanbul-lib-instrument": "^3.1.0", - "istanbul-lib-report": "^2.0.4", - "istanbul-lib-source-maps": "^3.0.2", - "istanbul-reports": "^2.1.1", - "js-yaml": "^3.12.0", - "make-dir": "^1.3.0", + "istanbul-lib-coverage": "^2.0.4", + "istanbul-lib-hook": "^2.0.6", + "istanbul-lib-instrument": "^3.2.0", + "istanbul-lib-report": "^2.0.7", + "istanbul-lib-source-maps": "^3.0.5", + "istanbul-reports": "^2.2.3", + "js-yaml": "^3.13.0", + "make-dir": "^2.1.0", "minimatch": "^3.0.4", "once": "^1.4.0" } }, "istanbul-lib-coverage": { - "version": "2.0.3", - "resolved": "https://registry.npmjs.org/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.3.tgz", - "integrity": "sha512-dKWuzRGCs4G+67VfW9pBFFz2Jpi4vSp/k7zBcJ888ofV5Mi1g5CUML5GvMvV6u9Cjybftu+E8Cgp+k0dI1E5lw==", + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/istanbul-lib-coverage/-/istanbul-lib-coverage-2.0.4.tgz", + "integrity": "sha512-LXTBICkMARVgo579kWDm8SqfB6nvSDKNqIOBEjmJRnL04JvoMHCYGWaMddQnseJYtkEuEvO/sIcOxPLk9gERug==", "dev": true }, "istanbul-lib-hook": { - "version": "2.0.3", - "resolved": "https://registry.npmjs.org/istanbul-lib-hook/-/istanbul-lib-hook-2.0.3.tgz", - "integrity": "sha512-CLmEqwEhuCYtGcpNVJjLV1DQyVnIqavMLFHV/DP+np/g3qvdxu3gsPqYoJMXm15sN84xOlckFB3VNvRbf5yEgA==", + "version": "2.0.6", + "resolved": "https://registry.npmjs.org/istanbul-lib-hook/-/istanbul-lib-hook-2.0.6.tgz", + "integrity": "sha512-829DKONApZ7UCiPXcOYWSgkFXa4+vNYoNOt3F+4uDJLKL1OotAoVwvThoEj1i8jmOj7odbYcR3rnaHu+QroaXg==", "dev": true, "requires": { "append-transform": "^1.0.0" } }, "istanbul-lib-instrument": { - "version": "3.1.0", - "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-3.1.0.tgz", - "integrity": "sha512-ooVllVGT38HIk8MxDj/OIHXSYvH+1tq/Vb38s8ixt9GoJadXska4WkGY+0wkmtYCZNYtaARniH/DixUGGLZ0uA==", + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/istanbul-lib-instrument/-/istanbul-lib-instrument-3.2.0.tgz", + "integrity": "sha512-06IM3xShbNW4NgZv5AP4QH0oHqf1/ivFo8eFys0ZjPXHGldHJQWb3riYOKXqmOqfxXBfxu4B+g/iuhOPZH0RJg==", "dev": true, "requires": { "@babel/generator": "^7.0.0", @@ -5544,30 +5549,38 @@ "@babel/template": "^7.0.0", "@babel/traverse": "^7.0.0", "@babel/types": "^7.0.0", - "istanbul-lib-coverage": "^2.0.3", - "semver": "^5.5.0" + "istanbul-lib-coverage": "^2.0.4", + "semver": "^6.0.0" + }, + "dependencies": { + "semver": { + "version": "6.0.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-6.0.0.tgz", + "integrity": "sha512-0UewU+9rFapKFnlbirLi3byoOuhrSsli/z/ihNnvM24vgF+8sNBiI1LZPBSH9wJKUwaUbw+s3hToDLCXkrghrQ==", + "dev": true + } } }, "istanbul-lib-report": { - "version": "2.0.4", - "resolved": "https://registry.npmjs.org/istanbul-lib-report/-/istanbul-lib-report-2.0.4.tgz", - "integrity": "sha512-sOiLZLAWpA0+3b5w5/dq0cjm2rrNdAfHWaGhmn7XEFW6X++IV9Ohn+pnELAl9K3rfpaeBfbmH9JU5sejacdLeA==", + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/istanbul-lib-report/-/istanbul-lib-report-2.0.7.tgz", + "integrity": "sha512-wLH6beJBFbRBLiTlMOBxmb85cnVM1Vyl36N48e4e/aTKSM3WbOx7zbVIH1SQ537fhhsPbX0/C5JB4qsmyRXXyA==", "dev": true, "requires": { - "istanbul-lib-coverage": "^2.0.3", - "make-dir": "^1.3.0", + "istanbul-lib-coverage": "^2.0.4", + "make-dir": "^2.1.0", "supports-color": "^6.0.0" } }, "istanbul-lib-source-maps": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/istanbul-lib-source-maps/-/istanbul-lib-source-maps-3.0.2.tgz", - "integrity": "sha512-JX4v0CiKTGp9fZPmoxpu9YEkPbEqCqBbO3403VabKjH+NRXo72HafD5UgnjTEqHL2SAjaZK1XDuDOkn6I5QVfQ==", + "version": "3.0.5", + "resolved": "https://registry.npmjs.org/istanbul-lib-source-maps/-/istanbul-lib-source-maps-3.0.5.tgz", + "integrity": "sha512-eDhZ7r6r1d1zQPVZehLc3D0K14vRba/eBYkz3rw16DLOrrTzve9RmnkcwrrkWVgO1FL3EK5knujVe5S8QHE9xw==", "dev": true, "requires": { "debug": "^4.1.1", - "istanbul-lib-coverage": "^2.0.3", - "make-dir": "^1.3.0", + "istanbul-lib-coverage": "^2.0.4", + "make-dir": "^2.1.0", "rimraf": "^2.6.2", "source-map": "^0.6.1" }, @@ -5584,9 +5597,9 @@ } }, "istanbul-reports": { - "version": "2.1.1", - "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-2.1.1.tgz", - "integrity": "sha512-FzNahnidyEPBCI0HcufJoSEoKykesRlFcSzQqjH9x0+LC8tnnE/p/90PBLu8iZTxr8yYZNyTtiAujUqyN+CIxw==", + "version": "2.2.3", + "resolved": "https://registry.npmjs.org/istanbul-reports/-/istanbul-reports-2.2.3.tgz", + "integrity": "sha512-T6EbPuc8Cb620LWAYyZ4D8SSn06dY9i1+IgUX2lTH8gbwflMc9Obd33zHTyNX653ybjpamAHS9toKS3E6cGhTw==", "dev": true, "requires": { "handlebars": "^4.1.0" @@ -6009,9 +6022,9 @@ "integrity": "sha512-RdJUflcE3cUzKiMqQgsCu06FPu9UdIJO0beYbPhHN4k6apgJtifcoCtT9bcxOpYBtpD2kCM6Sbzg4CausW/PKQ==" }, "js-yaml": { - "version": "3.12.1", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.12.1.tgz", - "integrity": "sha512-um46hB9wNOKlwkHgiuyEVAybXBjwFUV0Z/RaHJblRd9DXltue9FTYvzCr9ErQrK9Adz5MU4gHWVaNUfdmrC8qA==", + "version": "3.13.1", + "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.13.1.tgz", + "integrity": "sha512-YfbcO7jXDdyj0DGxYVSlSeQNHbD7XPWvrVWeVUujrQEoZzWJIRrCPoyk6kL6IAjAG2IolMK4T0hNUe0HOUs5Jw==", "dev": true, "requires": { "argparse": "^1.0.7", @@ -6058,6 +6071,12 @@ "xml-name-validator": "^3.0.0" } }, + "jsesc": { + "version": "2.5.2", + "resolved": "https://registry.npmjs.org/jsesc/-/jsesc-2.5.2.tgz", + "integrity": "sha512-OYu7XEzjkCQ3C5Ps3QIZsQfNpqoJyZZA99wd9aWd05NCtC5pWOkShK2mkL6HXQR6/Cy2lbNdPlZBpuQHXE63gA==", + "dev": true + }, "json-parse-better-errors": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/json-parse-better-errors/-/json-parse-better-errors-1.0.2.tgz", @@ -6131,9 +6150,9 @@ "dev": true }, "kleur": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/kleur/-/kleur-3.0.2.tgz", - "integrity": "sha512-3h7B2WRT5LNXOtQiAaWonilegHcPSf9nLVXlSTci8lu1dZUuui61+EsPEZqSVxY7rXYmB2DVKMQILxaO5WL61Q==", + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/kleur/-/kleur-3.0.3.tgz", + "integrity": "sha512-eTIzlVOSUR+JxdDFepEYcBMtZ9Qqdef+rnzWdRZuMbOywu5tO2w2N7rqjoANZ5k9vywhL6Br1VRjUIgTQx4E8w==", "dev": true }, "known-css-properties": { @@ -6189,18 +6208,6 @@ "spdx-expression-parse": "^3.0.0", "spdx-satisfies": "^4.0.0", "treeify": "^1.1.0" - }, - "dependencies": { - "nopt": { - "version": "4.0.1", - "resolved": "https://registry.npmjs.org/nopt/-/nopt-4.0.1.tgz", - "integrity": "sha1-0NRoWv1UFRk8jHUFYC0NF81kR00=", - "dev": true, - "requires": { - "abbrev": "1", - "osenv": "^0.1.4" - } - } } }, "load-json-file": { @@ -6368,12 +6375,21 @@ } }, "make-dir": { - "version": "1.3.0", - "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-1.3.0.tgz", - "integrity": "sha512-2w31R7SJtieJJnQtGc7RVL2StM2vGYVfqUOvUDxH6bC6aJTxPxTF0GnIgCyu7tjockiUWAYQRbxa7vKn34s5sQ==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/make-dir/-/make-dir-2.1.0.tgz", + "integrity": "sha512-LS9X+dc8KLxXCb8dni79fLIIUA5VyZoyjSMCwTluaXA0o27cCK0bhXkpgw+sTXVpPy/lSO57ilRixqk0vDmtRA==", "dev": true, "requires": { - "pify": "^3.0.0" + "pify": "^4.0.1", + "semver": "^5.6.0" + }, + "dependencies": { + "pify": { + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/pify/-/pify-4.0.1.tgz", + "integrity": "sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g==", + "dev": true + } } }, "make-error": { @@ -6472,13 +6488,13 @@ "dev": true }, "mem": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/mem/-/mem-4.1.0.tgz", - "integrity": "sha512-I5u6Q1x7wxO0kdOpYBB28xueHADYps5uty/zg936CiG8NTe5sJL8EjrCuLneuDW3PlMdZBGDIn8BirEVdovZvg==", + "version": "4.3.0", + "resolved": "https://registry.npmjs.org/mem/-/mem-4.3.0.tgz", + "integrity": "sha512-qX2bG48pTqYRVmDB37rn/6PT7LcR8T7oAX3bf99u1Tt1nzxYfxkgqDwUwolPlXweM0XzBOBFzSx4kfp7KP1s/w==", "dev": true, "requires": { "map-age-cleaner": "^0.1.1", - "mimic-fn": "^1.0.0", + "mimic-fn": "^2.0.0", "p-is-promise": "^2.0.0" } }, @@ -6665,24 +6681,24 @@ "dev": true }, "mime-db": { - "version": "1.38.0", - "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.38.0.tgz", - "integrity": "sha512-bqVioMFFzc2awcdJZIzR3HjZFX20QhilVS7hytkKrv7xFAn8bM1gzc/FOX2awLISvWe0PV8ptFKcon+wZ5qYkg==", + "version": "1.40.0", + "resolved": "https://registry.npmjs.org/mime-db/-/mime-db-1.40.0.tgz", + "integrity": "sha512-jYdeOMPy9vnxEqFRRo6ZvTZ8d9oPb+k18PKoYNYUe2stVEBPPwsln/qWzdbmaIvnhZ9v2P+CuecK+fpUfsV2mA==", "dev": true }, "mime-types": { - "version": "2.1.22", - "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.22.tgz", - "integrity": "sha512-aGl6TZGnhm/li6F7yx82bJiBZwgiEa4Hf6CNr8YO+r5UHr53tSTYZb102zyU50DOWWKeOv0uQLRL0/9EiKWCog==", + "version": "2.1.24", + "resolved": "https://registry.npmjs.org/mime-types/-/mime-types-2.1.24.tgz", + "integrity": "sha512-WaFHS3MCl5fapm3oLxU4eYDw77IQM2ACcxQ9RIxfaC3ooc6PFuBMGZZsYpvoXS5D5QTWPieo1jjLdAm3TBP3cQ==", "dev": true, "requires": { - "mime-db": "~1.38.0" + "mime-db": "1.40.0" } }, "mimic-fn": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-1.2.0.tgz", - "integrity": "sha512-jf84uxzwiuiIVKiOLpfYk7N46TSy8ubTonmneY9vrpHNAnp0QBt2BxWV9dO3/j+BoVAb+a5G6YDPW3M5HOdMWQ==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/mimic-fn/-/mimic-fn-2.1.0.tgz", + "integrity": "sha512-OqbOk5oEQeAZ8WXWydlu9HJjz9WVdEIvamMCcXmuqUYjTknH/sqsWvhQ3vgwKFRR1HpjvNBKQ37nbJgYzGqGcg==", "dev": true }, "minimalistic-assert": { @@ -6796,110 +6812,6 @@ } } }, - "mocha": { - "version": "6.1.3", - "resolved": "https://registry.npmjs.org/mocha/-/mocha-6.1.3.tgz", - "integrity": "sha512-QdE/w//EPHrqgT5PNRUjRVHy6IJAzAf1R8n2O8W8K2RZ+NbPfOD5cBDp+PGa2Gptep37C/TdBiaNwakppEzEbg==", - "dev": true, - "requires": { - "ansi-colors": "3.2.3", - "browser-stdout": "1.3.1", - "debug": "3.2.6", - "diff": "3.5.0", - "escape-string-regexp": "1.0.5", - "find-up": "3.0.0", - "glob": "7.1.3", - "growl": "1.10.5", - "he": "1.2.0", - "js-yaml": "3.13.0", - "log-symbols": "2.2.0", - "minimatch": "3.0.4", - "mkdirp": "0.5.1", - "ms": "2.1.1", - "node-environment-flags": "1.0.5", - "object.assign": "4.1.0", - "strip-json-comments": "2.0.1", - "supports-color": "6.0.0", - "which": "1.3.1", - "wide-align": "1.1.3", - "yargs": "13.2.2", - "yargs-parser": "13.0.0", - "yargs-unparser": "1.5.0" - }, - "dependencies": { - "get-caller-file": { - "version": "2.0.5", - "resolved": "https://registry.npmjs.org/get-caller-file/-/get-caller-file-2.0.5.tgz", - "integrity": "sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==", - "dev": true - }, - "js-yaml": { - "version": "3.13.0", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.13.0.tgz", - "integrity": "sha512-pZZoSxcCYco+DIKBTimr67J6Hy+EYGZDY/HCWC+iAEA9h1ByhMXAIVUXMcMFpOCxQ/xjXmPI2MkDL5HRm5eFrQ==", - "dev": true, - "requires": { - "argparse": "^1.0.7", - "esprima": "^4.0.0" - } - }, - "require-main-filename": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-2.0.0.tgz", - "integrity": "sha512-NKN5kMDylKuldxYLSUfrbo5Tuzh4hd+2E8NPPX02mZtn1VuREQToYe/ZdlJy+J3uCpfaiGF05e7B8W0iXbQHmg==", - "dev": true - }, - "string-width": { - "version": "3.1.0", - "resolved": "https://registry.npmjs.org/string-width/-/string-width-3.1.0.tgz", - "integrity": "sha512-vafcv6KjVZKSgz06oM/H6GDBrAtz8vdhQakGjFIvNrHA6y3HCF1CInLy+QLq8dTJPQ1b+KDUqDFctkdRW44e1w==", - "dev": true, - "requires": { - "emoji-regex": "^7.0.1", - "is-fullwidth-code-point": "^2.0.0", - "strip-ansi": "^5.1.0" - } - }, - "supports-color": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-6.0.0.tgz", - "integrity": "sha512-on9Kwidc1IUQo+bQdhi8+Tijpo0e1SS6RoGo2guUwn5vdaxw8RXOF9Vb2ws+ihWOmh4JnCJOvaziZWP1VABaLg==", - "dev": true, - "requires": { - "has-flag": "^3.0.0" - } - }, - "yargs": { - "version": "13.2.2", - "resolved": "https://registry.npmjs.org/yargs/-/yargs-13.2.2.tgz", - "integrity": "sha512-WyEoxgyTD3w5XRpAQNYUB9ycVH/PQrToaTXdYXRdOXvEy1l19br+VJsc0vcO8PTGg5ro/l/GY7F/JMEBmI0BxA==", - "dev": true, - "requires": { - "cliui": "^4.0.0", - "find-up": "^3.0.0", - "get-caller-file": "^2.0.1", - "os-locale": "^3.1.0", - "require-directory": "^2.1.1", - "require-main-filename": "^2.0.0", - "set-blocking": "^2.0.0", - "string-width": "^3.0.0", - "which-module": "^2.0.0", - "y18n": "^4.0.0", - "yargs-parser": "^13.0.0" - } - }, - "yargs-parser": { - "version": "13.0.0", - "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-13.0.0.tgz", - "integrity": "sha512-w2LXjoL8oRdRQN+hOyppuXs+V/fVAYtpcrRxZuF7Kt/Oc+Jr2uAcVntaUTNT6w5ihoWfFDpNY8CPx1QskxZ/pw==", - "dev": true, - "requires": { - "camelcase": "^5.0.0", - "decamelize": "^1.2.0" - } - } - } - }, "move-concurrently": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/move-concurrently/-/move-concurrently-1.0.1.tgz", @@ -6936,9 +6848,9 @@ "dev": true }, "nan": { - "version": "2.12.1", - "resolved": "https://registry.npmjs.org/nan/-/nan-2.12.1.tgz", - "integrity": "sha512-JY7V6lRkStKcKTvHO5NVSQRv+RV+FIL5pvDoLiAtSL9pKlC5x9PKQcZDsq7m4FO4d57mkhC6Z+QhAh3Jdk5JFw==", + "version": "2.13.2", + "resolved": "https://registry.npmjs.org/nan/-/nan-2.13.2.tgz", + "integrity": "sha512-TghvYc72wlMGMVMluVo9WRJc0mB8KxxF/gZ4YYFy7V2ZQX9l7rgbPg7vjS9mt6U5HXODVFVI2bOduCzwOMv/lw==", "dev": true }, "nanomatch": { @@ -6984,24 +6896,6 @@ "integrity": "sha512-1nh45deeb5olNY7eX82BkPO7SSxR5SSYJiPTrTdFUVYwAl8CKMA5N9PjTYkHiRjisVcxcQ1HXdLhx2qxxJzLNQ==", "dev": true }, - "node-environment-flags": { - "version": "1.0.5", - "resolved": "https://registry.npmjs.org/node-environment-flags/-/node-environment-flags-1.0.5.tgz", - "integrity": "sha512-VNYPRfGfmZLx0Ye20jWzHUjyTW/c+6Wq+iLhDzUI4XmhrDd9l/FozXV3F2xOaXjvp0co0+v1YSR3CMP6g+VvLQ==", - "dev": true, - "requires": { - "object.getownpropertydescriptors": "^2.0.3", - "semver": "^5.7.0" - }, - "dependencies": { - "semver": { - "version": "5.7.0", - "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.0.tgz", - "integrity": "sha512-Ya52jSX2u7QKghxeoFGpLwCtGlt7j0oY9DYb5apt9nPlJ42ID+ulTXESnt/qAQcoSERyZ5sl3LDIOw0nAn/5DA==", - "dev": true - } - } - }, "node-fetch": { "version": "1.7.3", "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-1.7.3.tgz", @@ -7037,6 +6931,15 @@ "which": "1" }, "dependencies": { + "nopt": { + "version": "3.0.6", + "resolved": "https://registry.npmjs.org/nopt/-/nopt-3.0.6.tgz", + "integrity": "sha1-xkZdvwirzU2zWTF/eaxopkayj/k=", + "dev": true, + "requires": { + "abbrev": "1" + } + }, "semver": { "version": "5.3.0", "resolved": "https://registry.npmjs.org/semver/-/semver-5.3.0.tgz", @@ -7110,9 +7013,9 @@ } }, "node-releases": { - "version": "1.1.13", - "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-1.1.13.tgz", - "integrity": "sha512-fKZGviSXR6YvVPyc011NHuJDSD8gFQvLPmc2d2V3BS4gr52ycyQ1Xzs7a8B+Ax3Ni/W+5h1h4SqmzeoA8WZRmA==", + "version": "1.1.15", + "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-1.1.15.tgz", + "integrity": "sha512-cKV097BQaZr8LTSRUa2+oc/aX5L8UkZtPQrMSTgiJEeaW7ymTDCoRaGCoaTqk0lqnalcoSHu4wjSl0Cmj2+bMw==", "dev": true, "requires": { "semver": "^5.3.0" @@ -7214,12 +7117,13 @@ } }, "nopt": { - "version": "3.0.6", - "resolved": "https://registry.npmjs.org/nopt/-/nopt-3.0.6.tgz", - "integrity": "sha1-xkZdvwirzU2zWTF/eaxopkayj/k=", + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/nopt/-/nopt-4.0.1.tgz", + "integrity": "sha1-0NRoWv1UFRk8jHUFYC0NF81kR00=", "dev": true, "requires": { - "abbrev": "1" + "abbrev": "1", + "osenv": "^0.1.4" } }, "normalize-package-data": { @@ -7235,13 +7139,10 @@ } }, "normalize-path": { - "version": "2.1.1", - "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-2.1.1.tgz", - "integrity": "sha1-GrKLVW4Zg2Oowab35vogE3/mrtk=", - "dev": true, - "requires": { - "remove-trailing-separator": "^1.0.1" - } + "version": "3.0.0", + "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", + "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", + "dev": true }, "normalize-range": { "version": "0.1.2", @@ -7347,9 +7248,9 @@ } }, "object-keys": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.1.0.tgz", - "integrity": "sha512-6OO5X1+2tYkNyNEx6TsCxEqFfRWaqx6EtMiSbGrw8Ob8v9Ne+Hl8rBAgLBZn5wjEz3s/s6U1WXFUFOcxxAwUpg==" + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/object-keys/-/object-keys-1.1.1.tgz", + "integrity": "sha512-NuAESUOUMrlIXOfHKzD6bpPu3tYt3xvjNdRIQ+FeT0lNb4K8WR70CaDxhuNguS2XG+GjkyMwOzsN5ZktImfhLA==" }, "object-visit": { "version": "1.0.1", @@ -7360,18 +7261,6 @@ "isobject": "^3.0.0" } }, - "object.assign": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/object.assign/-/object.assign-4.1.0.tgz", - "integrity": "sha512-exHJeq6kBKj58mqGyTQ9DFvrZC/eR6OwxzoM9YRoGBqrXYonaFyGiFMuc9VZrXf7DarreEwMpurG3dd+CNyW5w==", - "dev": true, - "requires": { - "define-properties": "^1.1.2", - "function-bind": "^1.1.1", - "has-symbols": "^1.0.0", - "object-keys": "^1.0.11" - } - }, "object.entries": { "version": "1.1.0", "resolved": "https://registry.npmjs.org/object.entries/-/object.entries-1.1.0.tgz", @@ -7442,6 +7331,12 @@ "wrappy": "1" } }, + "opener": { + "version": "1.5.1", + "resolved": "https://registry.npmjs.org/opener/-/opener-1.5.1.tgz", + "integrity": "sha512-goYSy5c2UXE4Ra1xixabeVh1guIX/ZV/YokJksb6q2lubWu6UbvPQ20p542/sFIll1nl8JnCyK9oBaOcCWXwvA==", + "dev": true + }, "opn": { "version": "5.5.0", "resolved": "https://registry.npmjs.org/opn/-/opn-5.5.0.tgz", @@ -7561,15 +7456,15 @@ "dev": true }, "p-is-promise": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/p-is-promise/-/p-is-promise-2.0.0.tgz", - "integrity": "sha512-pzQPhYMCAgLAKPWD2jC3Se9fEfrD9npNos0y150EeqZll7akhEgGhTW/slB6lHku8AvYGiJ+YJ5hfHKePPgFWg==", + "version": "2.1.0", + "resolved": "https://registry.npmjs.org/p-is-promise/-/p-is-promise-2.1.0.tgz", + "integrity": "sha512-Y3W0wlRPK8ZMRbNq97l4M5otioeA5lm1z7bkNkxCka8HSPjR0xRWmpCmc9utiaLP9Jb1eD8BgeIxTW4AIF45Pg==", "dev": true }, "p-limit": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.1.0.tgz", - "integrity": "sha512-NhURkNcrVB+8hNfLuysU8enY5xn2KXphsHBaC2YmRNTZRc7RWusw6apSpdEj3jo4CMb6W9nrF6tTnsJsJeyu6g==", + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/p-limit/-/p-limit-2.2.0.tgz", + "integrity": "sha512-pZbTJpoUsCzV48Mc9Nh51VbwO0X9cuPFE8gYwx9BTCt9SF8/b7Zljd2fVgOxhIF/HDTKgpVzs+GPhyKfjLLFRQ==", "dev": true, "requires": { "p-try": "^2.0.0" @@ -7597,9 +7492,9 @@ "dev": true }, "p-try": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/p-try/-/p-try-2.0.0.tgz", - "integrity": "sha512-hMp0onDKIajHfIkdRk3P4CdCmErkYAxxDtP3Wx/4nZ3aGlau2VKh3mZpcuFkH27WQkL/3WBCPOktzA9ZOAnMQQ==", + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/p-try/-/p-try-2.2.0.tgz", + "integrity": "sha512-R4nPAVTAU0B9D35/Gk3uJf/7XYbQcyohSKdvAxIRSNghFl4e71hVoGnBNQz9cWaXxO2I10KTC+3jMdvvoKw6dQ==", "dev": true }, "pako": { @@ -7670,9 +7565,9 @@ "dev": true }, "parseurl": { - "version": "1.3.2", - "resolved": "https://registry.npmjs.org/parseurl/-/parseurl-1.3.2.tgz", - "integrity": "sha1-/CidTtiZMRlGDBViUyYs3I3mW/M=", + "version": "1.3.3", + "resolved": "https://registry.npmjs.org/parseurl/-/parseurl-1.3.3.tgz", + "integrity": "sha512-CiyeOxFT/JZyN5m0z9PfXw4SCBJ6Sygz1Dpl0wqjlhDEGGBP1GnsUVEL0p63hoG1fcj3fHynXi9NYO4nWOL+qQ==", "dev": true }, "pascalcase": { @@ -7868,6 +7763,25 @@ "requires": { "postcss": "^7.0.2", "postcss-selector-parser": "^5.0.0" + }, + "dependencies": { + "cssesc": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-2.0.0.tgz", + "integrity": "sha512-MsCAG1z9lPdoO/IUMLSBWBSVxVtJ1395VGIQ+Fc2gNdkQ1hNDnQdw3YhA71WJCBW1vdwA0cAnk/DnW6bqoEUYg==", + "dev": true + }, + "postcss-selector-parser": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-5.0.0.tgz", + "integrity": "sha512-w+zLE5Jhg6Liz8+rQOWEAwtwkyqpfnmsinXjXg6cY7YIONZZtgvE0v2O0uhQBs0peNomOJwWRKt6JBfTdTd3OQ==", + "dev": true, + "requires": { + "cssesc": "^2.0.0", + "indexes-of": "^1.0.1", + "uniq": "^1.0.1" + } + } } }, "postcss-cli": { @@ -7977,6 +7891,25 @@ "requires": { "postcss": "^7.0.2", "postcss-selector-parser": "^5.0.0-rc.3" + }, + "dependencies": { + "cssesc": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-2.0.0.tgz", + "integrity": "sha512-MsCAG1z9lPdoO/IUMLSBWBSVxVtJ1395VGIQ+Fc2gNdkQ1hNDnQdw3YhA71WJCBW1vdwA0cAnk/DnW6bqoEUYg==", + "dev": true + }, + "postcss-selector-parser": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-5.0.0.tgz", + "integrity": "sha512-w+zLE5Jhg6Liz8+rQOWEAwtwkyqpfnmsinXjXg6cY7YIONZZtgvE0v2O0uhQBs0peNomOJwWRKt6JBfTdTd3OQ==", + "dev": true, + "requires": { + "cssesc": "^2.0.0", + "indexes-of": "^1.0.1", + "uniq": "^1.0.1" + } + } } }, "postcss-dir-pseudo-class": { @@ -7987,6 +7920,25 @@ "requires": { "postcss": "^7.0.2", "postcss-selector-parser": "^5.0.0-rc.3" + }, + "dependencies": { + "cssesc": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-2.0.0.tgz", + "integrity": "sha512-MsCAG1z9lPdoO/IUMLSBWBSVxVtJ1395VGIQ+Fc2gNdkQ1hNDnQdw3YhA71WJCBW1vdwA0cAnk/DnW6bqoEUYg==", + "dev": true + }, + "postcss-selector-parser": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-5.0.0.tgz", + "integrity": "sha512-w+zLE5Jhg6Liz8+rQOWEAwtwkyqpfnmsinXjXg6cY7YIONZZtgvE0v2O0uhQBs0peNomOJwWRKt6JBfTdTd3OQ==", + "dev": true, + "requires": { + "cssesc": "^2.0.0", + "indexes-of": "^1.0.1", + "uniq": "^1.0.1" + } + } } }, "postcss-double-position-gradients": { @@ -8177,19 +8129,6 @@ "postcss": "^7.0.6", "postcss-selector-parser": "^6.0.0", "postcss-value-parser": "^3.3.1" - }, - "dependencies": { - "postcss-selector-parser": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-6.0.2.tgz", - "integrity": "sha512-36P2QR59jDTOAiIkqEprfJDsoNrvwFei3eCqKd1Y0tUsBimsq39BLp7RD+JWny3WgB1zGhJX8XVePwm9k4wdBg==", - "dev": true, - "requires": { - "cssesc": "^3.0.0", - "indexes-of": "^1.0.1", - "uniq": "^1.0.1" - } - } } }, "postcss-modules-scope": { @@ -8200,19 +8139,6 @@ "requires": { "postcss": "^7.0.6", "postcss-selector-parser": "^6.0.0" - }, - "dependencies": { - "postcss-selector-parser": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-6.0.2.tgz", - "integrity": "sha512-36P2QR59jDTOAiIkqEprfJDsoNrvwFei3eCqKd1Y0tUsBimsq39BLp7RD+JWny3WgB1zGhJX8XVePwm9k4wdBg==", - "dev": true, - "requires": { - "cssesc": "^3.0.0", - "indexes-of": "^1.0.1", - "uniq": "^1.0.1" - } - } } }, "postcss-modules-values": { @@ -8315,6 +8241,25 @@ "requires": { "postcss": "^7.0.2", "postcss-selector-parser": "^5.0.0-rc.3" + }, + "dependencies": { + "cssesc": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-2.0.0.tgz", + "integrity": "sha512-MsCAG1z9lPdoO/IUMLSBWBSVxVtJ1395VGIQ+Fc2gNdkQ1hNDnQdw3YhA71WJCBW1vdwA0cAnk/DnW6bqoEUYg==", + "dev": true + }, + "postcss-selector-parser": { + "version": "5.0.0", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-5.0.0.tgz", + "integrity": "sha512-w+zLE5Jhg6Liz8+rQOWEAwtwkyqpfnmsinXjXg6cY7YIONZZtgvE0v2O0uhQBs0peNomOJwWRKt6JBfTdTd3OQ==", + "dev": true, + "requires": { + "cssesc": "^2.0.0", + "indexes-of": "^1.0.1", + "uniq": "^1.0.1" + } + } } }, "postcss-replace-overflow-wrap": { @@ -8393,22 +8338,14 @@ } }, "postcss-selector-parser": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-5.0.0.tgz", - "integrity": "sha512-w+zLE5Jhg6Liz8+rQOWEAwtwkyqpfnmsinXjXg6cY7YIONZZtgvE0v2O0uhQBs0peNomOJwWRKt6JBfTdTd3OQ==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/postcss-selector-parser/-/postcss-selector-parser-6.0.2.tgz", + "integrity": "sha512-36P2QR59jDTOAiIkqEprfJDsoNrvwFei3eCqKd1Y0tUsBimsq39BLp7RD+JWny3WgB1zGhJX8XVePwm9k4wdBg==", "dev": true, "requires": { - "cssesc": "^2.0.0", + "cssesc": "^3.0.0", "indexes-of": "^1.0.1", "uniq": "^1.0.1" - }, - "dependencies": { - "cssesc": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/cssesc/-/cssesc-2.0.0.tgz", - "integrity": "sha512-MsCAG1z9lPdoO/IUMLSBWBSVxVtJ1395VGIQ+Fc2gNdkQ1hNDnQdw3YhA71WJCBW1vdwA0cAnk/DnW6bqoEUYg==", - "dev": true - } } }, "postcss-syntax": { @@ -8450,14 +8387,6 @@ "ansi-regex": "^4.0.0", "ansi-styles": "^3.2.0", "react-is": "^16.8.4" - }, - "dependencies": { - "react-is": { - "version": "16.8.6", - "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.8.6.tgz", - "integrity": "sha512-aUk3bHfZ2bRSVFFbbeVS4i+lNPZr3/WM5jT2J5omUVV1zzcs1nAaf3l51ctA5FFvCRbhrH0bdAsRRQddFJZPtA==", - "dev": true - } } }, "pretty-hrtime": { @@ -8513,13 +8442,13 @@ } }, "proxy-addr": { - "version": "2.0.4", - "resolved": "https://registry.npmjs.org/proxy-addr/-/proxy-addr-2.0.4.tgz", - "integrity": "sha512-5erio2h9jp5CHGwcybmxmVqHmnCBZeewlfJ0pex+UW7Qny7OOZXTtH56TGNyBizkgiOwhJtMKrVzDTeKcySZwA==", + "version": "2.0.5", + "resolved": "https://registry.npmjs.org/proxy-addr/-/proxy-addr-2.0.5.tgz", + "integrity": "sha512-t/7RxHXPH6cJtP0pRG6smSr9QJidhB+3kXu0KgXnbGYMgzEnUxRQ4/LDdfOwZEMyIh3/xHb8PX3t+lfL9z+YVQ==", "dev": true, "requires": { "forwarded": "~0.1.2", - "ipaddr.js": "1.8.0" + "ipaddr.js": "1.9.0" } }, "prr": { @@ -8683,9 +8612,9 @@ } }, "react-ace": { - "version": "6.4.0", - "resolved": "https://registry.npmjs.org/react-ace/-/react-ace-6.4.0.tgz", - "integrity": "sha512-woTTgGk9x4GRRWiM4QLNOspjaJAYLX3UZ3J2XRYQvJiN6wyxrFY9x7rdOKc+4Tj+khb/ccPiDj/kll4UeJEDPw==", + "version": "6.5.0", + "resolved": "https://registry.npmjs.org/react-ace/-/react-ace-6.5.0.tgz", + "integrity": "sha512-W8iA6669Tf3sfjCsBg8gKs2pUVMy6BroX6O6GZcgadnLN+MTq7jhs6Q2Rsjq3E3SrWjyA9vZgs1Uzjy8XgWX5w==", "requires": { "brace": "^0.11.1", "diff-match-patch": "^1.0.4", @@ -8694,6 +8623,15 @@ "prop-types": "^15.6.2" } }, + "react-copy-to-clipboard": { + "version": "5.0.1", + "resolved": "https://registry.npmjs.org/react-copy-to-clipboard/-/react-copy-to-clipboard-5.0.1.tgz", + "integrity": "sha512-ELKq31/E3zjFs5rDWNCfFL4NvNFQvGRoJdAKReD/rUPA+xxiLPQmZBZBvy2vgH7V0GE9isIQpT9WXbwIVErYdA==", + "requires": { + "copy-to-clipboard": "^3", + "prop-types": "^15.5.8" + } + }, "react-dom": { "version": "16.8.6", "resolved": "https://registry.npmjs.org/react-dom/-/react-dom-16.8.6.tgz", @@ -8706,9 +8644,9 @@ } }, "react-is": { - "version": "16.8.3", - "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.8.3.tgz", - "integrity": "sha512-Y4rC1ZJmsxxkkPuMLwvKvlL1Zfpbcu+Bf4ZigkHup3v9EfdYhAlWAaVyA19olXq2o2mGn0w+dFKvk3pVVlYcIA==" + "version": "16.8.6", + "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.8.6.tgz", + "integrity": "sha512-aUk3bHfZ2bRSVFFbbeVS4i+lNPZr3/WM5jT2J5omUVV1zzcs1nAaf3l51ctA5FFvCRbhrH0bdAsRRQddFJZPtA==" }, "react-lifecycles-compat": { "version": "3.0.4", @@ -8918,9 +8856,9 @@ } }, "regenerator-runtime": { - "version": "0.12.1", - "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.12.1.tgz", - "integrity": "sha512-odxIc1/vDlo4iZcfXqRYFj0vpXFNoGdKMAUieAlFYO6m/nl5e9KR/beGf41z4a1FI+aQgtjhuaSlDxQ0hmkrHg==" + "version": "0.13.2", + "resolved": "https://registry.npmjs.org/regenerator-runtime/-/regenerator-runtime-0.13.2.tgz", + "integrity": "sha512-S/TQAZJO+D3m9xeN1WTI8dLKBBiRgXBlTJvbWjCThHWZj9EvHK70Ff50/tYj2J/fvBY6JtFVwRuazHN2E7M9BA==" }, "regex-not": { "version": "1.0.2", @@ -9100,9 +9038,9 @@ "dev": true }, "require-main-filename": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-1.0.1.tgz", - "integrity": "sha1-l/cXtp1IeE9fUmpsWqj/3aBVpNE=", + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-2.0.0.tgz", + "integrity": "sha512-NKN5kMDylKuldxYLSUfrbo5Tuzh4hd+2E8NPPX02mZtn1VuREQToYe/ZdlJy+J3uCpfaiGF05e7B8W0iXbQHmg==", "dev": true }, "requires-port": { @@ -9142,6 +9080,32 @@ "requires": { "expand-tilde": "^2.0.0", "global-modules": "^1.0.0" + }, + "dependencies": { + "global-modules": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/global-modules/-/global-modules-1.0.0.tgz", + "integrity": "sha512-sKzpEkf11GpOFuw0Zzjzmt4B4UZwjOcG757PPvrfhxcLFbq0wpsgpOqxpxtxFiCG4DtG93M6XRVbF2oGdev7bg==", + "dev": true, + "requires": { + "global-prefix": "^1.0.1", + "is-windows": "^1.0.1", + "resolve-dir": "^1.0.0" + } + }, + "global-prefix": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/global-prefix/-/global-prefix-1.0.2.tgz", + "integrity": "sha1-2/dDxsFJklk8ZVVoy2btMsASLr4=", + "dev": true, + "requires": { + "expand-tilde": "^2.0.2", + "homedir-polyfill": "^1.0.1", + "ini": "^1.3.4", + "is-windows": "^1.0.1", + "which": "^1.2.14" + } + } } }, "resolve-from": { @@ -9385,6 +9349,12 @@ "read-pkg": "^1.0.0" } }, + "require-main-filename": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-1.0.1.tgz", + "integrity": "sha1-l/cXtp1IeE9fUmpsWqj/3aBVpNE=", + "dev": true + }, "string-width": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/string-width/-/string-width-1.0.2.tgz", @@ -9535,9 +9505,9 @@ } }, "semver": { - "version": "5.6.0", - "resolved": "https://registry.npmjs.org/semver/-/semver-5.6.0.tgz", - "integrity": "sha512-RS9R6R35NYgQn++fkDWaOmqGoj4Ek9gGs+DPxNUZKuwE183xjJroKvyo1IzVFeXvUrvmALy6FWD5xrdJT25gMg==", + "version": "5.7.0", + "resolved": "https://registry.npmjs.org/semver/-/semver-5.7.0.tgz", + "integrity": "sha512-Ya52jSX2u7QKghxeoFGpLwCtGlt7j0oY9DYb5apt9nPlJ42ID+ulTXESnt/qAQcoSERyZ5sl3LDIOw0nAn/5DA==", "dev": true }, "send": { @@ -9579,9 +9549,9 @@ } }, "serialize-javascript": { - "version": "1.6.1", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-1.6.1.tgz", - "integrity": "sha512-A5MOagrPFga4YaKQSWHryl7AXvbQkEqpw4NNYMTNYUNV51bA8ABHgYFpqKx+YFFrw59xMV1qGH1R4AgoNIVgCw==", + "version": "1.7.0", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-1.7.0.tgz", + "integrity": "sha512-ke8UG8ulpFOxO8f8gRYabHQe/ZntKlcig2Mp+8+URDP1D8vJZ0KUt7LYo07q25Z/+JVSgpr/cui9PIp5H6/+nA==", "dev": true }, "serve-index": { @@ -9942,9 +9912,9 @@ } }, "source-map-support": { - "version": "0.5.11", - "resolved": "https://registry.npmjs.org/source-map-support/-/source-map-support-0.5.11.tgz", - "integrity": "sha512-//sajEx/fGL3iw6fltKMdPvy8kL3kJ2O3iuYlRoT3k9Kb4BjOoZ+BZzaNHeuaruSt+Kf3Zk9tnfAQg9/AJqUVQ==", + "version": "0.5.12", + "resolved": "https://registry.npmjs.org/source-map-support/-/source-map-support-0.5.12.tgz", + "integrity": "sha512-4h2Pbvyy15EE02G+JOZpUCmqWJuqrs+sEkzewTm++BPi7Hvn/HwcqLAcNxYAyI0x13CpPPn+kMjl+hplXMHITQ==", "dev": true, "requires": { "buffer-from": "^1.0.0", @@ -9995,9 +9965,9 @@ } }, "spdx-license-ids": { - "version": "3.0.3", - "resolved": "https://registry.npmjs.org/spdx-license-ids/-/spdx-license-ids-3.0.3.tgz", - "integrity": "sha512-uBIcIl3Ih6Phe3XHK1NqboJLdGfwr1UN3k6wSD1dZpmPsIkb8AGNbZYJ1fOBk834+Gxy8rpfDxrS6XLEMZMY2g==", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/spdx-license-ids/-/spdx-license-ids-3.0.4.tgz", + "integrity": "sha512-7j8LYJLeY/Yb6ACbQ7F76qy5jHkp0U6jgBfJsk97bwWlVUnUWsAgpyaCvo17h0/RQGnQ036tVDomiwoI4pDkQA==", "dev": true }, "spdx-ranges": { @@ -10370,12 +10340,6 @@ "get-stdin": "^4.0.1" } }, - "strip-json-comments": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-2.0.1.tgz", - "integrity": "sha1-PFMZQukIwml8DsNEhYwobHygpgo=", - "dev": true - }, "style-loader": { "version": "0.23.1", "resolved": "https://registry.npmjs.org/style-loader/-/style-loader-0.23.1.tgz", @@ -10447,12 +10411,6 @@ "table": "^5.0.0" }, "dependencies": { - "ansi-regex": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-4.1.0.tgz", - "integrity": "sha512-1apePfXM1UOSqw0o9IiFAovVz9M5S1Dg+4TrDwfMewQ6p/rmMueb7tWZjQ1rx4Loy1ArBggoqGpfqqdI4rondg==", - "dev": true - }, "camelcase": { "version": "4.1.0", "resolved": "https://registry.npmjs.org/camelcase/-/camelcase-4.1.0.tgz", @@ -10506,30 +10464,10 @@ "integrity": "sha512-jp4tHawyV7+fkkSKyvjuLZswblUtz+SQKzSWnBbii16BuZksJlU1wuBYXY75r+duh/llF1ur6oNwi+2ZzjKZ7g==", "dev": true }, - "global-modules": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/global-modules/-/global-modules-2.0.0.tgz", - "integrity": "sha512-NGbfmJBp9x8IxyJSd1P+otYK8vonoJactOogrVfFRIAEY1ukil8RSKDz2Yo7wh1oihl51l/r6W4epkeKJHqL8A==", - "dev": true, - "requires": { - "global-prefix": "^3.0.0" - } - }, - "global-prefix": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/global-prefix/-/global-prefix-3.0.0.tgz", - "integrity": "sha512-awConJSVCHVGND6x3tmMaKcQvwXLhjdkmomy2W+Goaui8YPgYgXJZewhg3fWC+DlfqqQuWg8AwqjGTD2nAPVWg==", - "dev": true, - "requires": { - "ini": "^1.3.5", - "kind-of": "^6.0.2", - "which": "^1.3.1" - } - }, "ignore": { - "version": "5.0.6", - "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.0.6.tgz", - "integrity": "sha512-/+hp3kUf/Csa32ktIaj0OlRqQxrgs30n62M90UBpNd9k+ENEch5S+hmbW3DtcJGz3sYFTh4F3A6fQ0q7KWsp4w==", + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/ignore/-/ignore-5.1.1.tgz", + "integrity": "sha512-DWjnQIFLenVrwyRCKZT+7a7/U4Cqgar4WG8V++K3hw+lrW1hc/SIwdiGmtxKCVACmHULTuGeBbHJmbwW7/sAvA==", "dev": true }, "indent-string": { @@ -10538,16 +10476,6 @@ "integrity": "sha1-Sl/W0nzDMvN+VBmlBNu4NxBckok=", "dev": true }, - "js-yaml": { - "version": "3.13.0", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.13.0.tgz", - "integrity": "sha512-pZZoSxcCYco+DIKBTimr67J6Hy+EYGZDY/HCWC+iAEA9h1ByhMXAIVUXMcMFpOCxQ/xjXmPI2MkDL5HRm5eFrQ==", - "dev": true, - "requires": { - "argparse": "^1.0.7", - "esprima": "^4.0.0" - } - }, "locate-path": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/locate-path/-/locate-path-2.0.0.tgz", @@ -10659,15 +10587,6 @@ "strip-ansi": "^5.1.0" } }, - "strip-ansi": { - "version": "5.2.0", - "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-5.2.0.tgz", - "integrity": "sha512-DuRs1gKbBqsMKIZlrffwlug8MHkcnpjs5VPmL1PAh+mA30U0DTotfDZ0d2UUsXpPmPmMMJ6W773MaA3J+lbiWA==", - "dev": true, - "requires": { - "ansi-regex": "^4.1.0" - } - }, "strip-indent": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/strip-indent/-/strip-indent-2.0.0.tgz", @@ -10692,30 +10611,30 @@ } }, "stylelint-config-recommended": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/stylelint-config-recommended/-/stylelint-config-recommended-2.1.0.tgz", - "integrity": "sha512-ajMbivOD7JxdsnlS5945KYhvt7L/HwN6YeYF2BH6kE4UCLJR0YvXMf+2j7nQpJyYLZx9uZzU5G1ZOSBiWAc6yA==", + "version": "2.2.0", + "resolved": "https://registry.npmjs.org/stylelint-config-recommended/-/stylelint-config-recommended-2.2.0.tgz", + "integrity": "sha512-bZ+d4RiNEfmoR74KZtCKmsABdBJr4iXRiCso+6LtMJPw5rd/KnxUWTxht7TbafrTJK1YRjNgnN0iVZaJfc3xJA==", "dev": true }, "stylelint-config-recommended-scss": { - "version": "3.2.0", - "resolved": "https://registry.npmjs.org/stylelint-config-recommended-scss/-/stylelint-config-recommended-scss-3.2.0.tgz", - "integrity": "sha512-M8BFHMRf8KNz5EQPKJd8nMCGmBd2o5coDEObfHVbEkyLDgjIf1V+U5dHjaGgvhm0zToUxshxN+Gc5wpbOOew4g==", + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/stylelint-config-recommended-scss/-/stylelint-config-recommended-scss-3.3.0.tgz", + "integrity": "sha512-BvuuLYwoet8JutOP7K1a8YaiENN+0HQn390eDi0SWe1h7Uhx6O3GUQ6Ubgie9b/AmHX4Btmp+ZzVGbzriFTBcA==", "dev": true, "requires": { - "stylelint-config-recommended": "^2.0.0" + "stylelint-config-recommended": "^2.2.0" } }, "stylelint-scss": { - "version": "3.5.4", - "resolved": "https://registry.npmjs.org/stylelint-scss/-/stylelint-scss-3.5.4.tgz", - "integrity": "sha512-hEdEOfFXVqxWcUbenBONW/cAw5cJcEDasY8tGwKNAAn1GDHoZO1ATdWpr+iIk325mPGIQqVb1sUxsRxuL70trw==", + "version": "3.6.0", + "resolved": "https://registry.npmjs.org/stylelint-scss/-/stylelint-scss-3.6.0.tgz", + "integrity": "sha512-Qpw0gl6iLBon5JNeFZjVYOEayd/e+WYIdY2vFhZuXeHC6jb8wl0wRZY97jATt/uxZzdtU3tGLAvJOUMuFp18vw==", "dev": true, "requires": { "lodash": "^4.17.11", "postcss-media-query-parser": "^0.2.3", "postcss-resolve-nested-selector": "^0.1.1", - "postcss-selector-parser": "^5.0.0", + "postcss-selector-parser": "^6.0.2", "postcss-value-parser": "^3.3.1" } }, @@ -10806,12 +10725,6 @@ "string-width": "^3.0.0" }, "dependencies": { - "ansi-regex": { - "version": "4.1.0", - "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-4.1.0.tgz", - "integrity": "sha512-1apePfXM1UOSqw0o9IiFAovVz9M5S1Dg+4TrDwfMewQ6p/rmMueb7tWZjQ1rx4Loy1ArBggoqGpfqqdI4rondg==", - "dev": true - }, "string-width": { "version": "3.1.0", "resolved": "https://registry.npmjs.org/string-width/-/string-width-3.1.0.tgz", @@ -10822,22 +10735,13 @@ "is-fullwidth-code-point": "^2.0.0", "strip-ansi": "^5.1.0" } - }, - "strip-ansi": { - "version": "5.2.0", - "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-5.2.0.tgz", - "integrity": "sha512-DuRs1gKbBqsMKIZlrffwlug8MHkcnpjs5VPmL1PAh+mA30U0DTotfDZ0d2UUsXpPmPmMMJ6W773MaA3J+lbiWA==", - "dev": true, - "requires": { - "ansi-regex": "^4.1.0" - } } } }, "tapable": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/tapable/-/tapable-1.1.1.tgz", - "integrity": "sha512-9I2ydhj8Z9veORCw5PRm4u9uebCn0mcCa6scWoNcbZ6dAtoo2618u9UUzxgmsCOreJpqDDuv61LvwofW7hLcBA==", + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/tapable/-/tapable-1.1.3.tgz", + "integrity": "sha512-4WK/bYZmj8xLr+HUCODHGF1ZFzsYffasLUgEiMBY4fgtltdO6B4WJtlSbPaDTLpYTcGVwM2qLnFTICEcNxs3kA==", "dev": true }, "tar": { @@ -10879,15 +10783,15 @@ } }, "test-exclude": { - "version": "5.1.0", - "resolved": "https://registry.npmjs.org/test-exclude/-/test-exclude-5.1.0.tgz", - "integrity": "sha512-gwf0S2fFsANC55fSeSqpb8BYk6w3FDvwZxfNjeF6FRgvFa43r+7wRiA/Q0IxoRU37wB/LE8IQ4221BsNucTaCA==", + "version": "5.2.2", + "resolved": "https://registry.npmjs.org/test-exclude/-/test-exclude-5.2.2.tgz", + "integrity": "sha512-N2pvaLpT8guUpb5Fe1GJlmvmzH3x+DAKmmyEQmFP792QcLYoGE1syxztSvPD1V8yPe6VrcCt6YGQVjSRjCASsA==", "dev": true, "requires": { - "arrify": "^1.0.1", + "glob": "^7.1.3", "minimatch": "^3.0.4", "read-pkg-up": "^4.0.0", - "require-main-filename": "^1.0.1" + "require-main-filename": "^2.0.0" } }, "throat": { @@ -10991,6 +10895,11 @@ "repeat-string": "^1.6.1" } }, + "toggle-selection": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/toggle-selection/-/toggle-selection-1.0.6.tgz", + "integrity": "sha1-bkWxJj8gF/oKzH2J14sVuL932jI=" + }, "tough-cookie": { "version": "2.5.0", "resolved": "https://registry.npmjs.org/tough-cookie/-/tough-cookie-2.5.0.tgz", @@ -11055,6 +10964,12 @@ "glob": "^7.1.2" } }, + "tryer": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/tryer/-/tryer-1.0.1.tgz", + "integrity": "sha512-c3zayb8/kWWpycWYg87P71E1S1ZL6b6IJxfb5fvsUgsf0S2MVGaDhDXXjDMpdCpfWXqptc+4mXwmiy1ypXqRAA==", + "dev": true + }, "ts-jest": { "version": "24.0.2", "resolved": "https://registry.npmjs.org/ts-jest/-/ts-jest-24.0.2.tgz", @@ -11099,9 +11014,9 @@ } }, "ts-loader": { - "version": "5.3.3", - "resolved": "https://registry.npmjs.org/ts-loader/-/ts-loader-5.3.3.tgz", - "integrity": "sha512-KwF1SplmOJepnoZ4eRIloH/zXL195F51skt7reEsS6jvDqzgc/YSbz9b8E07GxIUwLXdcD4ssrJu6v8CwaTafA==", + "version": "5.4.3", + "resolved": "https://registry.npmjs.org/ts-loader/-/ts-loader-5.4.3.tgz", + "integrity": "sha512-pHwZFkZioL7Yi2su0bhW2/djxZ+0iGat1cxlAif4Eg9j5znVYuWGtW0YYY/5w8W+IzLcAlD5KwJDrs5unUKIRA==", "dev": true, "requires": { "chalk": "^2.3.0", @@ -11112,9 +11027,9 @@ } }, "ts-node": { - "version": "8.0.3", - "resolved": "https://registry.npmjs.org/ts-node/-/ts-node-8.0.3.tgz", - "integrity": "sha512-2qayBA4vdtVRuDo11DEFSsD/SFsBXQBRZZhbRGSIkmYmVkWjULn/GGMdG10KVqkaGndljfaTD8dKjWgcejO8YA==", + "version": "8.1.0", + "resolved": "https://registry.npmjs.org/ts-node/-/ts-node-8.1.0.tgz", + "integrity": "sha512-34jpuOrxDuf+O6iW1JpgTRDFynUZ1iEqtYruBqh35gICNjN8x+LpVcPAcwzLPi9VU6mdA3ym+x233nZmZp445A==", "dev": true, "requires": { "arg": "^4.1.0", @@ -11130,12 +11045,12 @@ "integrity": "sha512-4krF8scpejhaOgqzBEcGM7yDIEfi0/8+8zDRZhNZZ2kjmHJ4hv3zCbQWxoJGz1iw5U0Jl0nma13xzHXcncMavQ==" }, "tslint": { - "version": "5.15.0", - "resolved": "https://registry.npmjs.org/tslint/-/tslint-5.15.0.tgz", - "integrity": "sha512-6bIEujKR21/3nyeoX2uBnE8s+tMXCQXhqMmaIPJpHmXJoBJPTLcI7/VHRtUwMhnLVdwLqqY3zmd8Dxqa5CVdJA==", + "version": "5.16.0", + "resolved": "https://registry.npmjs.org/tslint/-/tslint-5.16.0.tgz", + "integrity": "sha512-UxG2yNxJ5pgGwmMzPMYh/CCnCnh0HfPgtlVRDs1ykZklufFBL1ZoTlWFRz2NQjcoEiDoRp+JyT0lhBbbH/obyA==", "dev": true, "requires": { - "babel-code-frame": "^6.22.0", + "@babel/code-frame": "^7.0.0", "builtin-modules": "^1.1.1", "chalk": "^2.3.0", "commander": "^2.12.1", @@ -11148,18 +11063,6 @@ "semver": "^5.3.0", "tslib": "^1.8.0", "tsutils": "^2.29.0" - }, - "dependencies": { - "js-yaml": { - "version": "3.13.1", - "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-3.13.1.tgz", - "integrity": "sha512-YfbcO7jXDdyj0DGxYVSlSeQNHbD7XPWvrVWeVUujrQEoZzWJIRrCPoyk6kL6IAjAG2IolMK4T0hNUe0HOUs5Jw==", - "dev": true, - "requires": { - "argparse": "^1.0.7", - "esprima": "^4.0.0" - } - } } }, "tslint-loader": { @@ -11245,9 +11148,9 @@ "dev": true }, "typescript": { - "version": "3.4.3", - "resolved": "https://registry.npmjs.org/typescript/-/typescript-3.4.3.tgz", - "integrity": "sha512-FFgHdPt4T/duxx6Ndf7hwgMZZjZpB+U0nMNGVCYPq0rEzWKjEDobm4J6yb3CS7naZ0yURFqdw9Gwc7UOh/P9oQ==", + "version": "3.4.4", + "resolved": "https://registry.npmjs.org/typescript/-/typescript-3.4.4.tgz", + "integrity": "sha512-xt5RsIRCEaf6+j9AyOBgvVuAec0i92rgCaS3S+UVf5Z/vF2Hvtsw08wtUTJqp4djwznoAgjSxeCcU4r+CcDBJA==", "dev": true }, "ua-parser-js": { @@ -11256,13 +11159,13 @@ "integrity": "sha512-T3PVJ6uz8i0HzPxOF9SWzWAlfN/DavlpQqepn22xgve/5QecC+XMCAtmUNnY7C9StehaV6exjUCI801lOI7QlQ==" }, "uglify-js": { - "version": "3.5.3", - "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.5.3.tgz", - "integrity": "sha512-rIQPT2UMDnk4jRX+w4WO84/pebU2jiLsjgIyrCktYgSvx28enOE3iYQMr+BD1rHiitWnDmpu0cY/LfIEpKcjcw==", + "version": "3.5.6", + "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.5.6.tgz", + "integrity": "sha512-YDKRX8F0Y+Jr7LhoVk0n4G7ltR3Y7qFAj+DtVBthlOgCcIj1hyMigCfousVfn9HKmvJ+qiFlLDwaHx44/e5ZKw==", "dev": true, "optional": true, "requires": { - "commander": "~2.19.0", + "commander": "~2.20.0", "source-map": "~0.6.1" } }, @@ -11458,9 +11361,9 @@ } }, "upath": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/upath/-/upath-1.1.0.tgz", - "integrity": "sha512-bzpH/oBhoS/QI/YtbkqCg6VEiPYjSZtrHQM6/QnJS6OL9pKUFLqb3aFh4Scvwm45+7iAgiMkLhSbaZxUqmrprw==", + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/upath/-/upath-1.1.2.tgz", + "integrity": "sha512-kXpym8nmDmlCBr7nKdIx8P2jNBa+pBpIUFRnKJ4dr8htyYGJFokkr2ZvERRtUN+9SY+JqXouNgUPtv6JQva/2Q==", "dev": true }, "uri-js": { @@ -11497,9 +11400,9 @@ } }, "url-parse": { - "version": "1.4.5", - "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.4.5.tgz", - "integrity": "sha512-4XDvC5vZRjEpjP0L4znrWeoH8P8F0XGBlfLdABi/6oV4o8xUVbTpyrxWHxkK2bT0pSIpcjdIzSoWUhlUfawCAQ==", + "version": "1.4.6", + "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.4.6.tgz", + "integrity": "sha512-/B8AD9iQ01seoXmXf9z/MjLZQIdOoYl/+gvsQF6+mpnxaTfG9P7srYaiqaDMyKkR36XMXfhqSHss5MyFAO8lew==", "dev": true, "requires": { "querystringify": "^2.0.0", @@ -11690,9 +11593,9 @@ "dev": true }, "webpack": { - "version": "4.29.6", - "resolved": "https://registry.npmjs.org/webpack/-/webpack-4.29.6.tgz", - "integrity": "sha512-MwBwpiE1BQpMDkbnUUaW6K8RFZjljJHArC6tWQJoFm0oQtfoSebtg4Y7/QHnJ/SddtjYLHaKGX64CFjG5rehJw==", + "version": "4.30.0", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-4.30.0.tgz", + "integrity": "sha512-4hgvO2YbAFUhyTdlR4FNyt2+YaYBYHavyzjCMbZzgglo02rlKi/pcsEzwCuCpsn1ryzIl1cq/u8ArIKu8JBYMg==", "dev": true, "requires": { "@webassemblyjs/ast": "1.8.5", @@ -11729,10 +11632,48 @@ } } }, + "webpack-bundle-analyzer": { + "version": "3.3.2", + "resolved": "https://registry.npmjs.org/webpack-bundle-analyzer/-/webpack-bundle-analyzer-3.3.2.tgz", + "integrity": "sha512-7qvJLPKB4rRWZGjVp5U1KEjwutbDHSKboAl0IfafnrdXMrgC0tOtZbQD6Rw0u4cmpgRN4O02Fc0t8eAT+FgGzA==", + "dev": true, + "requires": { + "acorn": "^6.0.7", + "acorn-walk": "^6.1.1", + "bfj": "^6.1.1", + "chalk": "^2.4.1", + "commander": "^2.18.0", + "ejs": "^2.6.1", + "express": "^4.16.3", + "filesize": "^3.6.1", + "gzip-size": "^5.0.0", + "lodash": "^4.17.10", + "mkdirp": "^0.5.1", + "opener": "^1.5.1", + "ws": "^6.0.0" + }, + "dependencies": { + "acorn": { + "version": "6.1.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-6.1.1.tgz", + "integrity": "sha512-jPTiwtOxaHNaAPg/dmrJ/beuzLRnXtB0kQPQ8JpotKJgTB6rX6c8mlf315941pyjBSaPg8NHXS9fhP4u17DpGA==", + "dev": true + }, + "ws": { + "version": "6.2.1", + "resolved": "https://registry.npmjs.org/ws/-/ws-6.2.1.tgz", + "integrity": "sha512-GIyAXC2cB7LjvpgMt9EKS2ldqr0MTrORaleiOno6TweZ6r3TKtoFQWay/2PceJ3RuBasOHzXNn5Lrw1X0bEjqA==", + "dev": true, + "requires": { + "async-limiter": "~1.0.0" + } + } + } + }, "webpack-cli": { - "version": "3.3.0", - "resolved": "https://registry.npmjs.org/webpack-cli/-/webpack-cli-3.3.0.tgz", - "integrity": "sha512-t1M7G4z5FhHKJ92WRKwZ1rtvi7rHc0NZoZRbSkol0YKl4HvcC8+DsmGDmK7MmZxHSAetHagiOsjOB6MmzC2TUw==", + "version": "3.3.1", + "resolved": "https://registry.npmjs.org/webpack-cli/-/webpack-cli-3.3.1.tgz", + "integrity": "sha512-c2inFU7SM0IttEgF7fK6AaUsbBnORRzminvbyRKS+NlbQHVZdCtzKBlavRL5359bFsywXGRAItA5di/IruC8mg==", "dev": true, "requires": { "chalk": "^2.4.1", @@ -11748,6 +11689,30 @@ "yargs": "^12.0.5" }, "dependencies": { + "global-modules": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/global-modules/-/global-modules-1.0.0.tgz", + "integrity": "sha512-sKzpEkf11GpOFuw0Zzjzmt4B4UZwjOcG757PPvrfhxcLFbq0wpsgpOqxpxtxFiCG4DtG93M6XRVbF2oGdev7bg==", + "dev": true, + "requires": { + "global-prefix": "^1.0.1", + "is-windows": "^1.0.1", + "resolve-dir": "^1.0.0" + } + }, + "global-prefix": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/global-prefix/-/global-prefix-1.0.2.tgz", + "integrity": "sha1-2/dDxsFJklk8ZVVoy2btMsASLr4=", + "dev": true, + "requires": { + "expand-tilde": "^2.0.2", + "homedir-polyfill": "^1.0.1", + "ini": "^1.3.4", + "is-windows": "^1.0.1", + "which": "^1.2.14" + } + }, "supports-color": { "version": "5.5.0", "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.5.0.tgz", @@ -11823,26 +11788,6 @@ "integrity": "sha1-w7M6te42DYbg5ijwRorn7yfWVN8=", "dev": true }, - "chokidar": { - "version": "2.1.5", - "resolved": "https://registry.npmjs.org/chokidar/-/chokidar-2.1.5.tgz", - "integrity": "sha512-i0TprVWp+Kj4WRPtInjexJ8Q+BqTE909VpH8xVhXrJkoc5QC8VO9TryGOqTr+2hljzc1sC62t22h5tZePodM/A==", - "dev": true, - "requires": { - "anymatch": "^2.0.0", - "async-each": "^1.0.1", - "braces": "^2.3.2", - "fsevents": "^1.2.7", - "glob-parent": "^3.1.0", - "inherits": "^2.0.3", - "is-binary-path": "^1.0.0", - "is-glob": "^4.0.0", - "normalize-path": "^3.0.0", - "path-is-absolute": "^1.0.0", - "readdirp": "^2.2.1", - "upath": "^1.1.1" - } - }, "debug": { "version": "4.1.1", "resolved": "https://registry.npmjs.org/debug/-/debug-4.1.1.tgz", @@ -11852,12 +11797,6 @@ "ms": "^2.1.1" } }, - "normalize-path": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/normalize-path/-/normalize-path-3.0.0.tgz", - "integrity": "sha512-6eZs5Ls3WtCisHWp9S2GUy8dqkpGi4BVSz3GaqiE6ezub0512ESztXUwUB6C6IKbQkY2Pnb/mD4WYojCRwcwLA==", - "dev": true - }, "semver": { "version": "6.0.0", "resolved": "https://registry.npmjs.org/semver/-/semver-6.0.0.tgz", @@ -11872,12 +11811,6 @@ "requires": { "ansi-regex": "^2.0.0" } - }, - "upath": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/upath/-/upath-1.1.2.tgz", - "integrity": "sha512-kXpym8nmDmlCBr7nKdIx8P2jNBa+pBpIUFRnKJ4dr8htyYGJFokkr2ZvERRtUN+9SY+JqXouNgUPtv6JQva/2Q==", - "dev": true } } }, @@ -12117,6 +12050,14 @@ "which-module": "^2.0.0", "y18n": "^3.2.1 || ^4.0.0", "yargs-parser": "^11.1.1" + }, + "dependencies": { + "require-main-filename": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/require-main-filename/-/require-main-filename-1.0.1.tgz", + "integrity": "sha1-l/cXtp1IeE9fUmpsWqj/3aBVpNE=", + "dev": true + } } }, "yargs-parser": { @@ -12129,21 +12070,10 @@ "decamelize": "^1.2.0" } }, - "yargs-unparser": { - "version": "1.5.0", - "resolved": "https://registry.npmjs.org/yargs-unparser/-/yargs-unparser-1.5.0.tgz", - "integrity": "sha512-HK25qidFTCVuj/D1VfNiEndpLIeJN78aqgR23nL3y4N0U/91cOAzqfHlF8n2BvoNDcZmJKin3ddNSvOxSr8flw==", - "dev": true, - "requires": { - "flat": "^4.1.0", - "lodash": "^4.17.11", - "yargs": "^12.0.5" - } - }, "yn": { - "version": "3.0.0", - "resolved": "https://registry.npmjs.org/yn/-/yn-3.0.0.tgz", - "integrity": "sha512-+Wo/p5VRfxUgBUGy2j/6KX2mj9AYJWOHuhMjMcbBFc3y54o9/4buK1ksBvuiK01C3kby8DH9lSmJdSxw+4G/2Q==", + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/yn/-/yn-3.1.0.tgz", + "integrity": "sha512-kKfnnYkbTfrAdd0xICNFw7Atm8nKpLcLv9AZGEt+kczL/WQVai4e2V6ZN8U/O+iI6WrNuJjNNOyu4zfhl9D3Hg==", "dev": true } } From b66b875c42adad2266ee5d3e10a15965e16d391d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 12 May 2019 15:33:12 -0700 Subject: [PATCH 26/52] fix test --- .../actions/SegmentMetadataUpdateAction.java | 2 +- .../common/actions/SegmentNukeAction.java | 2 +- .../SegmentTransactionalInsertAction.java | 4 +- ...ctionPreconditions.java => TaskLocks.java} | 48 ++- .../druid/indexing/common/task/KillTask.java | 4 +- .../actions/TaskActionPreconditionsTest.java | 161 --------- .../common/actions/TaskLocksTest.java | 324 ++++++++++++++++++ 7 files changed, 375 insertions(+), 170 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/actions/{TaskActionPreconditions.java => TaskLocks.java} (71%) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java 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 fcfa74843a16..83e8bb96bb23 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 @@ -64,7 +64,7 @@ public TypeReference getReturnTypeReference() @Override public Void perform(Task task, TaskActionToolbox toolbox) { - TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); try { toolbox.getTaskLockbox().doInCriticalSection( 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 7b7d48605f30..4197a2167d3f 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 @@ -66,7 +66,7 @@ public TypeReference getReturnTypeReference() @Override public Void perform(Task task, TaskActionToolbox toolbox) { - TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), segments); try { toolbox.getTaskLockbox().doInCriticalSection( 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 269cf829e3fd..7f4207eb3c7f 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 @@ -141,7 +141,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) if (segmentsToBeOverwritten != null) { allSegments.addAll(segmentsToBeOverwritten); } - TaskActionPreconditions.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); + TaskLocks.checkLockCoversSegments(task, toolbox.getTaskLockbox(), allSegments); if (segmentsToBeOverwritten != null && !segmentsToBeOverwritten.isEmpty()) { final List locks = toolbox.getTaskLockbox().findLocksForTask(task); @@ -178,7 +178,7 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) } // Non-overwriting tasks should release locks as early as possible, so that other tasks can lock same segments. - final List locks = toolbox.getTaskLockbox().findLocksForTask(task); + final List locks = TaskLocks.findLocksForSegments(task, toolbox.getTaskLockbox(), segments); for (TaskLock lock : locks) { if (lock.getGranularity() == LockGranularity.SEGMENT) { final SegmentLock segmentLock = (SegmentLock) lock; 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/TaskLocks.java similarity index 71% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionPreconditions.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index 60d9264bb2a7..1e0847903a3b 100644 --- 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/TaskLocks.java @@ -32,12 +32,13 @@ 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 TaskActionPreconditions +public class TaskLocks { static void checkLockCoversSegments( final Task task, @@ -89,7 +90,6 @@ public static boolean isLockCoversSegments( } final List locks = entry.getValue(); - return locks.stream().anyMatch( lock -> { if (lock.getGranularity() == LockGranularity.TIME_CHUNK) { @@ -110,11 +110,53 @@ public static boolean isLockCoversSegments( ); } + public static List findLocksForSegments( + final Task task, + final TaskLockbox taskLockbox, + final Collection segments + ) + { + final NavigableMap> taskLockMap = getTaskLockMap(taskLockbox, task); + if (taskLockMap.isEmpty()) { + return Collections.emptyList(); + } + + final List found = new ArrayList<>(); + segments.forEach(segment -> { + final Entry> 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 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.getMajorVersion()) >= 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.getMajorVersion()) >= 0 + && segmentLock.getPartitionId() == segment.getShardSpec().getPartitionNum()) { + found.add(lock); + } + } + }); + }); + return found; + } + private static NavigableMap> getTaskLockMap(TaskLockbox taskLockbox, Task task) { final List taskLocks = taskLockbox.findLocksForTask(task); final NavigableMap> taskLockMap = new TreeMap<>(); - taskLocks.forEach(taskLock -> taskLockMap.computeIfAbsent(taskLock.getInterval().getStart(), k -> new ArrayList<>()).add(taskLock)); + 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/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java index d2aced9de75e..7e01e441267f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java @@ -27,7 +27,7 @@ import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; import org.apache.druid.indexing.common.actions.SegmentNukeAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.actions.TaskActionPreconditions; +import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; @@ -77,7 +77,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception .getTaskActionClient() .submit(new SegmentListUnusedAction(getDataSource(), getInterval())); - if (!TaskActionPreconditions.isLockCoversSegments(taskLockMap, unusedSegments)) { + if (!TaskLocks.isLockCoversSegments(taskLockMap, unusedSegments)) { throw new ISE( "Locks[%s] for task[%s] can't cover segments[%s]", taskLockMap.values().stream().flatMap(List::stream).collect(Collectors.toList()), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java deleted file mode 100644 index 2d5a753e19c5..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionPreconditionsTest.java +++ /dev/null @@ -1,161 +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.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.indexing.common.config.TaskStorageConfig; -import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; -import org.apache.druid.indexing.overlord.LockResult; -import org.apache.druid.indexing.overlord.TaskLockbox; -import org.apache.druid.indexing.overlord.TimeChunkLockRequest; -import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; - -public class TaskActionPreconditionsTest -{ - private TaskLockbox lockbox; - private Task task; - private Set segments; - - @Before - public void setup() - { - lockbox = new TaskLockbox( - new HeapMemoryTaskStorage(new TaskStorageConfig(null)), - new TestIndexerMetadataStorageCoordinator() - ); - task = NoopTask.create(); - lockbox.add(task); - - segments = ImmutableSet.of( - new DataSegment.Builder() - .dataSource(task.getDataSource()) - .interval(Intervals.of("2017-01-01/2017-01-02")) - .version(DateTimes.nowUtc().toString()) - .shardSpec(new LinearShardSpec(2)) - .build(), - new DataSegment.Builder() - .dataSource(task.getDataSource()) - .interval(Intervals.of("2017-01-02/2017-01-03")) - .version(DateTimes.nowUtc().toString()) - .shardSpec(new LinearShardSpec(2)) - .build(), - new DataSegment.Builder() - .dataSource(task.getDataSource()) - .interval(Intervals.of("2017-01-03/2017-01-04")) - .version(DateTimes.nowUtc().toString()) - .shardSpec(new LinearShardSpec(2)) - .build() - ); - } - - private LockResult tryTimeChunkLock(TaskLockType lockType, Task task, Interval interval) - { - return lockbox.tryLock(task, new TimeChunkLockRequest(lockType, task, interval, null)); - } - - @Test - public void testCheckLockCoversSegments() - { - final List intervals = ImmutableList.of( - Intervals.of("2017-01-01/2017-01-02"), - Intervals.of("2017-01-02/2017-01-03"), - Intervals.of("2017-01-03/2017-01-04") - ); - - final Map locks = intervals.stream().collect( - Collectors.toMap( - Function.identity(), - interval -> { - final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); - Assert.assertNotNull(lock); - return lock; - } - ) - ); - - Assert.assertEquals(3, locks.size()); - Assert.assertTrue(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); - } - - @Test - public void testCheckLargeLockCoversSegments() - { - final List intervals = ImmutableList.of( - Intervals.of("2017-01-01/2017-01-04") - ); - - final Map locks = intervals.stream().collect( - Collectors.toMap( - Function.identity(), - interval -> { - final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); - Assert.assertNotNull(lock); - return lock; - } - ) - ); - - Assert.assertEquals(1, locks.size()); - Assert.assertTrue(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); - } - - @Test - public void testCheckLockCoversSegmentsWithOverlappedIntervals() - { - final List lockIntervals = ImmutableList.of( - Intervals.of("2016-12-31/2017-01-01"), - Intervals.of("2017-01-01/2017-01-02"), - Intervals.of("2017-01-02/2017-01-03") - ); - - final Map locks = lockIntervals.stream().collect( - Collectors.toMap( - Function.identity(), - interval -> { - final TaskLock lock = tryTimeChunkLock(TaskLockType.EXCLUSIVE, task, interval).getTaskLock(); - Assert.assertNotNull(lock); - return lock; - } - ) - ); - - Assert.assertEquals(3, locks.size()); - Assert.assertFalse(TaskActionPreconditions.isLockCoversSegments(task, lockbox, segments)); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java new file mode 100644 index 000000000000..8b8f11ee8cf9 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java @@ -0,0 +1,324 @@ +/* + * 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.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.indexing.common.SegmentLock; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; +import org.apache.druid.indexing.overlord.LockResult; +import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.TimeChunkLockRequest; +import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class TaskLocksTest +{ + private TaskLockbox lockbox; + private Task task; + + @Before + public void setup() + { + lockbox = new TaskLockbox( + new HeapMemoryTaskStorage(new TaskStorageConfig(null)), + new TestIndexerMetadataStorageCoordinator() + ); + task = NoopTask.create(); + lockbox.add(task); + } + + private Set createTimeChunkedSegments() + { + return ImmutableSet.of( + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(DateTimes.nowUtc().toString()) + .shardSpec(new LinearShardSpec(2)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-02/2017-01-03")) + .version(DateTimes.nowUtc().toString()) + .shardSpec(new LinearShardSpec(2)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-03/2017-01-04")) + .version(DateTimes.nowUtc().toString()) + .shardSpec(new LinearShardSpec(2)) + .build() + ); + } + + private Set createNumberedPartitionedSegments() + { + final String version = DateTimes.nowUtc().toString(); + return ImmutableSet.of( + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(version) + .shardSpec(new NumberedShardSpec(0, 0)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(version) + .shardSpec(new NumberedShardSpec(1, 0)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(version) + .shardSpec(new NumberedShardSpec(2, 0)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(version) + .shardSpec(new NumberedShardSpec(3, 0)) + .build(), + new DataSegment.Builder() + .dataSource(task.getDataSource()) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(version) + .shardSpec(new NumberedShardSpec(4, 0)) + .build() + ); + } + + private LockResult tryTimeChunkLock(Task task, Interval interval) + { + return lockbox.tryLock(task, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task, interval, null)); + } + + private LockResult trySegmentLock(Task task, Interval interval, String version, int partitonId) + { + return lockbox.tryLock( + task, + new SpecificSegmentLockRequest(TaskLockType.EXCLUSIVE, task, interval, version, partitonId) + ); + } + + @Test + public void testCheckLockCoversSegments() + { + final Set segments = createTimeChunkedSegments(); + final List intervals = ImmutableList.of( + Intervals.of("2017-01-01/2017-01-02"), + Intervals.of("2017-01-02/2017-01-03"), + Intervals.of("2017-01-03/2017-01-04") + ); + + final Map locks = intervals.stream().collect( + Collectors.toMap( + Function.identity(), + interval -> { + final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); + Assert.assertNotNull(lock); + return lock; + } + ) + ); + + Assert.assertEquals(3, locks.size()); + Assert.assertTrue(TaskLocks.isLockCoversSegments(task, lockbox, segments)); + } + + @Test + public void testCheckSegmentLockCoversSegments() + { + final Set segments = createNumberedPartitionedSegments(); + final Interval interval = Intervals.of("2017-01-01/2017-01-02"); + + final String version = DateTimes.nowUtc().toString(); + final List locks = IntStream + .range(0, 5) + .mapToObj( + partitionId -> { + final TaskLock lock = trySegmentLock(task, interval, version, partitionId).getTaskLock(); + Assert.assertNotNull(lock); + return lock; + } + ).collect(Collectors.toList()); + + Assert.assertEquals(5, locks.size()); + Assert.assertTrue(TaskLocks.isLockCoversSegments(task, lockbox, segments)); + } + + @Test + public void testCheckLargeLockCoversSegments() + { + final Set segments = createTimeChunkedSegments(); + final List intervals = ImmutableList.of( + Intervals.of("2017-01-01/2017-01-04") + ); + + final Map locks = intervals.stream().collect( + Collectors.toMap( + Function.identity(), + interval -> { + final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); + Assert.assertNotNull(lock); + return lock; + } + ) + ); + + Assert.assertEquals(1, locks.size()); + Assert.assertTrue(TaskLocks.isLockCoversSegments(task, lockbox, segments)); + } + + @Test + public void testCheckLockCoversSegmentsWithOverlappedIntervals() + { + final Set segments = createTimeChunkedSegments(); + final List lockIntervals = ImmutableList.of( + Intervals.of("2016-12-31/2017-01-01"), + Intervals.of("2017-01-01/2017-01-02"), + Intervals.of("2017-01-02/2017-01-03") + ); + + final Map locks = lockIntervals.stream().collect( + Collectors.toMap( + Function.identity(), + interval -> { + final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); + Assert.assertNotNull(lock); + return lock; + } + ) + ); + + Assert.assertEquals(3, locks.size()); + Assert.assertFalse(TaskLocks.isLockCoversSegments(task, lockbox, segments)); + } + + @Test + public void testFindLocksForSegments() + { + final Set segments = createTimeChunkedSegments(); + final List intervals = ImmutableList.of( + Intervals.of("2017-01-01/2017-01-02"), + Intervals.of("2017-01-02/2017-01-03"), + Intervals.of("2017-01-03/2017-01-04") + ); + + final Map locks = intervals.stream().collect( + Collectors.toMap( + Function.identity(), + interval -> { + final TaskLock lock = tryTimeChunkLock(task, interval).getTaskLock(); + Assert.assertNotNull(lock); + return lock; + } + ) + ); + + Assert.assertEquals(3, locks.size()); + Assert.assertEquals( + ImmutableList.of( + newTimeChunkLock(intervals.get(0), locks.get(intervals.get(0)).getVersion()), + newTimeChunkLock(intervals.get(1), locks.get(intervals.get(1)).getVersion()), + newTimeChunkLock(intervals.get(2), locks.get(intervals.get(2)).getVersion()) + ), + TaskLocks.findLocksForSegments(task, lockbox, segments) + ); + } + + @Test + public void testFindSegmentLocksForSegments() + { + final Set segments = createNumberedPartitionedSegments(); + final Interval interval = Intervals.of("2017-01-01/2017-01-02"); + + final String version = DateTimes.nowUtc().toString(); + final List locks = IntStream + .range(0, 5) + .mapToObj( + partitionId -> { + final TaskLock lock = trySegmentLock(task, interval, version, partitionId).getTaskLock(); + Assert.assertNotNull(lock); + return lock; + } + ).collect(Collectors.toList()); + + Assert.assertEquals(5, locks.size()); + Assert.assertEquals( + ImmutableList.of( + newSegmentLock(interval, locks.get(0).getVersion(), 0), + newSegmentLock(interval, locks.get(0).getVersion(), 1), + newSegmentLock(interval, locks.get(0).getVersion(), 2), + newSegmentLock(interval, locks.get(0).getVersion(), 3), + newSegmentLock(interval, locks.get(0).getVersion(), 4) + ), + TaskLocks.findLocksForSegments(task, lockbox, segments) + ); + } + + private TimeChunkLock newTimeChunkLock(Interval interval, String version) + { + return new TimeChunkLock( + TaskLockType.EXCLUSIVE, + task.getGroupId(), + task.getDataSource(), + interval, + version, + task.getPriority() + ); + } + + private SegmentLock newSegmentLock(Interval interval, String version, int partitionId) + { + return new SegmentLock( + TaskLockType.EXCLUSIVE, + task.getGroupId(), + task.getDataSource(), + interval, + version, + partitionId, + task.getPriority() + ); + } +} From 79c43986f7395dac34f0fdbd5e2ccc54b1d591f8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 16 May 2019 14:00:10 -0700 Subject: [PATCH 27/52] fix style --- .../org/apache/druid/timeline/partition/PartitionIds.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java index 2c81236b510f..5275f1ddeaa0 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java @@ -27,5 +27,7 @@ public final class PartitionIds public static int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive public static short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; - private PartitionIds() {} + private PartitionIds() + { + } } From c0d8585b8e4fc3e56a9e271f7e22b35622a6d2d2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 20 May 2019 15:36:34 -0700 Subject: [PATCH 28/52] adding javadocs --- .../apache/druid/timeline/Overshadowable.java | 12 ++++++++++-- .../partition/NumberedOverwriteShardSpec.java | 5 ++++- .../NumberedOverwritingPartitionChunk.java | 3 +++ .../partition/OvershadowableManager.java | 19 +++++++++++++++---- .../timeline/partition/PartitionIds.java | 13 +++++++++++++ .../druid/timeline/partition/ShardSpec.java | 3 +++ .../timeline/partition/ShardSpecFactory.java | 13 +++++++++++++ .../indexing/common/LockGranularity.java | 3 +++ .../common/actions/SegmentAllocateAction.java | 7 ++++--- .../actions/SegmentLockAquireAction.java | 3 +++ .../actions/SegmentLockReleaseAction.java | 4 ++++ .../actions/SegmentLockTryAcquireAction.java | 3 +++ .../SegmentTransactionalInsertAction.java | 3 ++- .../common/task/AbstractBatchIndexTask.java | 7 ++++++- .../task/CachingLocalSegmentAllocator.java | 7 +++++-- .../common/task/CachingSegmentAllocator.java | 3 +++ .../druid/indexing/common/task/IndexTask.java | 5 ----- .../task/IndexTaskSegmentAllocator.java | 8 +++++--- 18 files changed, 99 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java index 2cdac31fcca6..90cd8f756c63 100644 --- a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java +++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java @@ -20,7 +20,7 @@ package org.apache.druid.timeline; /** - * Interface to check the overshadowing relation between objeccts. + * 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. * @@ -29,9 +29,17 @@ */ public interface Overshadowable { + /** + * Returns true this overshadows the given other. + */ default boolean isOvershadow(T other) { - return containsRootPartition(other) && getMinorVersion() > other.getMinorVersion(); + final int majorVersionCompare = getMajorVersion().compareTo(other.getMajorVersion()); + if (majorVersionCompare == 0) { + return containsRootPartition(other) && getMinorVersion() > other.getMinorVersion(); + } else { + return majorVersionCompare > 0; + } } default boolean containsRootPartition(T other) 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 index f36f4a6e046f..d5b36576a654 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java @@ -30,6 +30,9 @@ 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; @@ -216,7 +219,7 @@ public int hashCode() public String toString() { return "NumberedOverwriteShardSpec{" + - "partitionNum=" + partitionId + + "partitionId=" + partitionId + ", startRootPartitionId=" + startRootPartitionId + ", endRootPartitionId=" + endRootPartitionId + ", minorVersion=" + minorVersion + 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 index cc355092befc..0dfdb24880aa 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingPartitionChunk.java @@ -24,6 +24,9 @@ import java.util.Objects; +/** + * PartitionChunk corresponding to {@link NumberedOverwriteShardSpec} + */ public class NumberedOverwritingPartitionChunk implements PartitionChunk { private final int chunkId; 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 index 7d77a23d5cf6..3211dea6e5a9 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -52,10 +52,16 @@ import java.util.stream.Collectors; /** + * 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. * - * Not thread-safe + * This class is not thread-safe. */ -public class OvershadowableManager> +class OvershadowableManager> { private enum State { @@ -71,7 +77,7 @@ private enum State private final TreeMap>> visibleGroup; private final TreeMap>> overshadowedGroups; - public OvershadowableManager() + OvershadowableManager() { this.knownPartitionChunks = new HashMap<>(); this.standbyGroups = new TreeMap<>(); @@ -79,7 +85,7 @@ public OvershadowableManager() this.overshadowedGroups = new TreeMap<>(); } - public OvershadowableManager(OvershadowableManager other) + OvershadowableManager(OvershadowableManager other) { this.knownPartitionChunks = new HashMap<>(other.knownPartitionChunks); this.standbyGroups = new TreeMap<>(other.standbyGroups); @@ -575,6 +581,11 @@ public String toString() } } + /** + * 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 extends AbstractShort2ObjectSortedMap { private short key; diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java index 5275f1ddeaa0..155da827ec3c 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java @@ -21,10 +21,23 @@ public final class PartitionIds { + /** + * Start partitionId available for root generation segments. + */ public static int ROOT_GEN_START_PARTITION_ID = 0; + /** + * End partitionId available for root generation segments. + */ public static int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive + /** + * Start partitionId available for non-root generation segments. + */ public static int NON_ROOT_GEN_START_PARTITION_ID = 32768; + /** + * End partitionId available for non-root generation segments. + */ public static int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive + public static short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; private PartitionIds() 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 1adb26bb6319..43aaf701db36 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 @@ -83,5 +83,8 @@ default short getAtomicUpdateGroupSize() */ boolean possibleInDomain(Map> domain); + /** + * Returns true if two segments of this and other shardSpecs can exist in the same timeChunk. + */ boolean isCompatible(Class 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 index 7c520c742232..5b2c922e4e3a 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java @@ -25,6 +25,9 @@ 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), @@ -33,9 +36,19 @@ }) 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. + */ 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 getShardSpecClass(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java index bf71494d44cc..37cdecc91320 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/LockGranularity.java @@ -19,6 +19,9 @@ package org.apache.druid.indexing.common; +/** + * Granularity for {@link TaskLock}. + */ public enum LockGranularity { TIME_CHUNK, 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 5bc33f67292c..13cb7b19a245 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 @@ -53,10 +53,11 @@ * 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. *

- * This action implicitly acquires segment 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.) *

- * If this action cannot acquire an appropriate segment lock, or if it cannot expand an existing segment set, it returns + * 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 diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java index b9957090b590..76bf8c73ebce 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java @@ -29,6 +29,9 @@ import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest; import org.joda.time.Interval; +/** + * TaskAction to acquire a task lock for a segment. Used by stream ingestion tasks. + */ public class SegmentLockAquireAction implements TaskAction { private final TaskLockType lockType; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java index b4d4e95db4c2..6a47091710bd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockReleaseAction.java @@ -25,6 +25,10 @@ import org.apache.druid.indexing.common.task.Task; import org.joda.time.Interval; +/** + * 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 { private final Interval interval; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java index ed25ee41d08b..96f5e391c858 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java @@ -33,6 +33,9 @@ import java.util.Set; import java.util.stream.Collectors; +/** + * TaskAction to try to acquire a {@link org.apache.druid.indexing.common.SegmentLock}. + */ public class SegmentLockTryAcquireAction implements TaskAction> { private final TaskLockType type; 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 7f4207eb3c7f..443cc58b4b54 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 @@ -85,7 +85,7 @@ public static SegmentTransactionalInsertAction appendAction( @JsonCreator private SegmentTransactionalInsertAction( - @JsonProperty("segmentsToBeOverwritten") Set segmentsToBeOverwritten, + @JsonProperty("segmentsToBeOverwritten") @Nullable Set segmentsToBeOverwritten, @JsonProperty("segments") Set segments, @JsonProperty("startMetadata") @Nullable DataSourceMetadata startMetadata, @JsonProperty("endMetadata") @Nullable DataSourceMetadata endMetadata @@ -98,6 +98,7 @@ private SegmentTransactionalInsertAction( } @JsonProperty + @Nullable public Set getSegmentsToBeOverwritten() { return segmentsToBeOverwritten; 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 index 3d1bb0acb712..787636effc74 100644 --- 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 @@ -52,6 +52,11 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; +/** + * Abstract class for batch tasks like {@link IndexTask}. + * Provides some methods ({@link #tryLockWithIntervals} and {@link #tryLockWithSegments}) for easily acquiring task + * locks. + */ public abstract class AbstractBatchIndexTask extends AbstractTask { @Nullable @@ -151,7 +156,7 @@ protected boolean tryLockWithIntervals(TaskActionClient client, List i } } - boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException + private boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException { allInputSegments = Collections.emptySet(); overwritingRootGenPartitions = Collections.emptyMap(); 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 index 9e5a284eb75d..707372458491 100644 --- 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 @@ -37,9 +37,12 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -public class CachingLocalSegmentAllocator extends CachingSegmentAllocator +/** + * CachingSegmentAllocator which allocates segments locally. + */ +class CachingLocalSegmentAllocator extends CachingSegmentAllocator { - public CachingLocalSegmentAllocator( + CachingLocalSegmentAllocator( TaskToolbox toolbox, String taskId, String dataSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java index 708c513eab72..17d0fff53780 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java @@ -36,6 +36,9 @@ import java.util.List; import java.util.Map; +/** + * Allocates all necessary segments at the beginning and reuse them. + */ public abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocator { private final TaskToolbox toolbox; 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 ec7e4cf1ba05..c669b950827c 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 @@ -614,11 +614,6 @@ private Map getTaskCompletionRowStats() return metrics; } - public interface SequenceNameGenerator - { - String getSequenceName(Interval interval, InputRow inputRow); - } - private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) { Preconditions.checkState( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java index 4cdabb0446b0..2ebb9f1dad72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java @@ -19,12 +19,14 @@ package org.apache.druid.indexing.common.task; -import org.apache.druid.indexing.common.task.IndexTask.SequenceNameGenerator; +import org.apache.druid.data.input.InputRow; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; +import org.joda.time.Interval; /** - * Segment allocator interface for {@link IndexTask} + * Segment allocator interface for {@link IndexTask}. It has 3 different modes for allocating segments. */ -interface IndexTaskSegmentAllocator extends SegmentAllocator, SequenceNameGenerator +interface IndexTaskSegmentAllocator extends SegmentAllocator { + String getSequenceName(Interval interval, InputRow inputRow); } From cd4f3ead7e019a3b0cd87bc23e62a91fc024a634 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 20 May 2019 15:37:23 -0700 Subject: [PATCH 29/52] remove unused classes --- .../actions/SegmentBulkAllocateAction.java | 145 ------------------ .../task/CachingRemoteSegmentAllocator.java | 55 ------- 2 files changed, 200 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java deleted file mode 100644 index b934664433db..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentBulkAllocateAction.java +++ /dev/null @@ -1,145 +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.fasterxml.jackson.annotation.JsonCreator; -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.TaskLockType; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.LockRequest; -import org.apache.druid.indexing.overlord.LockRequestForNewSegment; -import org.apache.druid.indexing.overlord.LockResult; -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.ShardSpecFactory; -import org.joda.time.Interval; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -public class SegmentBulkAllocateAction implements TaskAction>> -{ - // interval -> # of segments to allocate - private final Map> allocateSpec; - private final String baseSequenceName; - - @JsonCreator - public SegmentBulkAllocateAction( - @JsonProperty("allocateSpec") Map> allocateSpec, - @JsonProperty("baseSequenceName") String baseSequenceName - ) - { - this.allocateSpec = allocateSpec; - this.baseSequenceName = baseSequenceName; - } - - @JsonProperty - public Map> getAllocateSpec() - { - return allocateSpec; - } - - @JsonProperty - public String getBaseSequenceName() - { - return baseSequenceName; - } - - @Override - public TypeReference>> getReturnTypeReference() - { - return new TypeReference>>() - { - }; - } - - @Override - public Map> perform(Task task, TaskActionToolbox toolbox) - { - final Map> segmentIds = new HashMap<>(allocateSpec.size()); - - for (Entry> 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 - ); - - for (int i = 0; i < numSegmentsToAllocate; i++) { - final String sequenceName = StringUtils.format("%s_%d", baseSequenceName, i); - final LockRequest lockRequest = new LockRequestForNewSegment( - TaskLockType.EXCLUSIVE, - task.getGroupId(), - task.getDataSource(), - interval, - shardSpecFactory, - task.getPriority(), - sequenceName, - null, - true - ); - - final LockResult lockResult = toolbox.getTaskLockbox().tryLock(task, lockRequest); - - if (lockResult.isRevoked()) { - // The lock was preempted by other tasks - throw new ISE("WTH? lock[%s] for new segment request is revoked?", lockResult.getTaskLock()); - } - - if (lockResult.isOk()) { - final SegmentIdWithShardSpec identifier = lockResult.getNewSegmentId(); - if (identifier != null) { - segmentIds.computeIfAbsent(interval, k -> new ArrayList<>()).add(identifier); - } else { - throw new ISE("Cannot allocate new pending segmentIds with request[%s]", lockRequest); - } - } else { - throw new ISE("Could not acquire lock with request[%s]", lockRequest); - } - } - } - - return segmentIds; - } - - @Override - public boolean isAudited() - { - return false; - } - - @Override - public String toString() - { - return "SegmentBulkAllocateAction{" + - "allocateSpec=" + allocateSpec + - ", baseSequenceName='" + baseSequenceName + '\'' + - '}'; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java deleted file mode 100644 index 5e96ddc6f3b4..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingRemoteSegmentAllocator.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.common.task; - -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentBulkAllocateAction; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.timeline.partition.ShardSpecFactory; -import org.joda.time.Interval; - -import java.io.IOException; -import java.util.List; -import java.util.Map; - -public class CachingRemoteSegmentAllocator extends CachingSegmentAllocator -{ - public CachingRemoteSegmentAllocator( - TaskToolbox toolbox, - String taskId, - String dataSource, - Map> allocateSpec - ) throws IOException - { - super(toolbox, taskId, dataSource, allocateSpec); - } - - @Override - Map> getIntervalToSegmentIds() throws IOException - { - return getToolbox().getTaskActionClient().submit( - new SegmentBulkAllocateAction( - getAllocateSpec(), - getTaskId() - ) - ); - } -} From 7611183fe777c79eaaa96b7291f565b819fbf60e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 20 May 2019 16:01:30 -0700 Subject: [PATCH 30/52] add more javadocs --- .../common/actions/SegmentTransactionalInsertAction.java | 9 --------- .../indexing/common/task/CachingSegmentAllocator.java | 4 ++-- .../indexing/common/task/LocalSegmentAllocator.java | 7 +++++-- .../indexing/common/task/RemoteSegmentAllocator.java | 7 +++++-- .../realtime/appenderator/BaseAppenderatorDriver.java | 2 +- .../realtime/appenderator/BatchAppenderatorDriver.java | 2 +- .../realtime/appenderator/StreamAppenderatorDriver.java | 6 +++--- ...va => StreamAppenderatorDriverSegmentLockHelper.java} | 5 ++++- 8 files changed, 21 insertions(+), 21 deletions(-) rename server/src/main/java/org/apache/druid/segment/realtime/appenderator/{SegmentLocker.java => StreamAppenderatorDriverSegmentLockHelper.java} (83%) 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 443cc58b4b54..b5156c7644d8 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 @@ -178,15 +178,6 @@ public SegmentPublishResult perform(Task task, TaskActionToolbox toolbox) throw new RuntimeException(e); } - // Non-overwriting tasks should release locks as early as possible, so that other tasks can lock same segments. - final List locks = TaskLocks.findLocksForSegments(task, toolbox.getTaskLockbox(), segments); - for (TaskLock lock : locks) { - if (lock.getGranularity() == LockGranularity.SEGMENT) { - final SegmentLock segmentLock = (SegmentLock) lock; - toolbox.getTaskLockbox().unlock(task, segmentLock.getInterval(), segmentLock.getPartitionId()); - } - } - // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java index 17d0fff53780..10d29fde5d61 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java @@ -39,7 +39,7 @@ /** * Allocates all necessary segments at the beginning and reuse them. */ -public abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocator +abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocator { private final TaskToolbox toolbox; private final String taskId; @@ -51,7 +51,7 @@ public abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocat // sequenceName -> segmentId private final Map sequenceNameToSegmentId; - public CachingSegmentAllocator( + CachingSegmentAllocator( TaskToolbox toolbox, String taskId, String dataSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java index cbaf7026e11d..44e63b05d034 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java @@ -39,13 +39,16 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -public class LocalSegmentAllocator implements IndexTaskSegmentAllocator +/** + * Segment allocator which allocates new segments locally per request. + */ +class LocalSegmentAllocator implements IndexTaskSegmentAllocator { private final String taskId; private final SegmentAllocator internalAllocator; - public LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec) + LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec) throws IOException { this.taskId = taskId; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java index 20e45c8957c2..d3ac959d31e0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java @@ -36,12 +36,15 @@ import java.io.IOException; import java.util.Map; -public class RemoteSegmentAllocator implements IndexTaskSegmentAllocator +/** + * Segment allocator which allocates new segments using the overlord per request. + */ +class RemoteSegmentAllocator implements IndexTaskSegmentAllocator { private final String taskId; private final ActionBasedSegmentAllocator internalAllocator; - public RemoteSegmentAllocator( + RemoteSegmentAllocator( final TaskToolbox toolbox, final String taskId, final DataSchema dataSchema, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index 25117b36604a..40df9a462b77 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -271,7 +271,7 @@ Map getSegments() * @return currently persisted commit metadata */ @Nullable - public abstract Object startJob(SegmentLocker segmentLocker); + public abstract Object startJob(StreamAppenderatorDriverSegmentLockHelper lockHelper); /** * Find a segment in the {@link SegmentState#APPENDING} state for the given timestamp and sequenceName. diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java index 8b6c7c536a8f..1264e004cd03 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java @@ -80,7 +80,7 @@ public BatchAppenderatorDriver( */ @Override @Nullable - public Object startJob(SegmentLocker segmentLocker) + public Object startJob(StreamAppenderatorDriverSegmentLockHelper lockHelper) { final Object metadata = appenderator.startJob(); if (metadata != null) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index b589b69e7590..918104fbd2a6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -107,7 +107,7 @@ public StreamAppenderatorDriver( @Override @Nullable - public Object startJob(SegmentLocker segmentLocker) + public Object startJob(StreamAppenderatorDriverSegmentLockHelper lockHelper) { handoffNotifier.start(); @@ -133,10 +133,10 @@ public Object startJob(SegmentLocker segmentLocker) final SegmentsForSequenceBuilder builder = new SegmentsForSequenceBuilder(lastSegmentIds.get(sequenceName)); builders.put(sequenceName, builder); entry.getValue().forEach(builder::add); - if (segmentLocker != null) { + if (lockHelper != null) { for (SegmentWithState segmentWithState : entry.getValue()) { if (segmentWithState.getState() != SegmentState.PUSHED_AND_DROPPED - && !segmentLocker.lock(segmentWithState.getSegmentIdentifier())) { + && !lockHelper.lock(segmentWithState.getSegmentIdentifier())) { throw new ISE("Failed to lock segment[%s]", segmentWithState.getSegmentIdentifier()); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentLocker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverSegmentLockHelper.java similarity index 83% rename from server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentLocker.java rename to server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverSegmentLockHelper.java index 6c5bcba3381d..ba1b44cd4db1 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentLocker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverSegmentLockHelper.java @@ -19,7 +19,10 @@ package org.apache.druid.segment.realtime.appenderator; -public interface SegmentLocker +/** + * Lock helper for {@link StreamAppenderatorDriver}. It's only used to lock segments on restart of the driver. + */ +public interface StreamAppenderatorDriverSegmentLockHelper { boolean lock(SegmentIdWithShardSpec segmentId); } From a08333c86b67f4f2dc7cf5f9eee3b453b6525a4d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 20 May 2019 16:25:25 -0700 Subject: [PATCH 31/52] unused import --- .../common/actions/SegmentTransactionalInsertAction.java | 1 - 1 file changed, 1 deletion(-) 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 b5156c7644d8..b585a80f326e 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 @@ -25,7 +25,6 @@ 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.SegmentLock; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.IndexTaskUtils; From 49d389704eba823f24613c0f1e0be09367dec9d2 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 20 May 2019 17:47:58 -0700 Subject: [PATCH 32/52] fix test --- .../org/apache/druid/client/CachingClusteredClientTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 45d7826e4f17..13238353ad85 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2833,7 +2833,7 @@ public Map getLoadSpec() @JsonProperty public String getMajorVersion() { - return baseSegment.getMajorVersion(); + return "version"; } @Override From 5f91f790a7efa5b7c2a611a2ba95328dd7643bc9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 28 May 2019 18:19:47 -0700 Subject: [PATCH 33/52] fix test --- .../org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 26107ba58498..f47289de5b1e 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 @@ -1660,7 +1660,7 @@ public void testRunOneTaskTwoPartitions() throws Exception SegmentDescriptor desc3 = sd("2011/P1D", 1); SegmentDescriptor desc4 = sd("2012/P1D", 0); final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableSet.of(desc1, desc2, desc4), publishedDescriptors()); + assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc4), publishedDescriptors()); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)) @@ -1681,7 +1681,7 @@ public void testRunOneTaskTwoPartitions() throws Exception // 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))) ); } From 194af327f319f3366a137afec46ca7e6c732fdac Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 29 Jun 2019 14:58:37 -0700 Subject: [PATCH 34/52] Support forceTimeChunk context and force timeChunk lock for parallel index task if intervals are missing --- .../common/granularity/AllGranularity.java | 4 +- .../granularity/DurationGranularity.java | 2 +- .../util/common/granularity/Granularity.java | 2 +- .../common/granularity/NoneGranularity.java | 4 +- .../common/granularity/PeriodGranularity.java | 2 +- .../apache/druid/timeline/Overshadowable.java | 10 - .../timeline/VersionedIntervalTimeline.java | 30 +- .../timeline/partition/AtomicUpdateGroup.java | 16 +- .../NumberedOverwritingShardSpecFactory.java | 2 + .../partition/OvershadowableManager.java | 61 +-- .../timeline/partition/PartitionHolder.java | 4 +- .../timeline/partition/ShardSpecFactory.java | 2 + ...ementalPublishingKafkaIndexTaskRunner.java | 7 +- .../druid/indexing/kafka/KafkaIndexTask.java | 3 +- .../indexing/kafka/KafkaIndexTaskTest.java | 21 + .../indexing/kinesis/KinesisIndexTask.java | 4 +- .../kinesis/KinesisIndexTaskRunner.java | 7 +- .../kinesis/KinesisIndexTaskTest.java | 22 + .../SegmentAllocateActionGenerator.java | 4 +- .../common/actions/SegmentAllocateAction.java | 14 +- ...ion.java => SegmentLockAcquireAction.java} | 12 +- .../actions/SegmentLockTryAcquireAction.java | 1 + .../SegmentTransactionalInsertAction.java | 6 +- .../indexing/common/actions/TaskAction.java | 2 +- .../actions/TimeChunkLockAcquireAction.java | 5 + .../TimeChunkLockTryAcquireAction.java | 4 + .../common/task/AbstractBatchIndexTask.java | 461 +++++++++--------- .../AppenderatorDriverRealtimeIndexTask.java | 8 +- .../task/CachingLocalSegmentAllocator.java | 95 +++- .../common/task/CachingSegmentAllocator.java | 131 ----- .../indexing/common/task/CompactionTask.java | 52 +- .../indexing/common/task/HadoopIndexTask.java | 22 +- .../druid/indexing/common/task/IndexTask.java | 128 +---- .../task/IndexTaskSegmentAllocator.java | 10 + .../common/task/LocalSegmentAllocator.java | 2 + .../common/task/RemoteSegmentAllocator.java | 71 ++- .../common/task/SegmentLockHelper.java | 349 +++++++++++++ .../druid/indexing/common/task/Tasks.java | 2 + .../batch/parallel/ParallelIndexSubTask.java | 94 ++-- .../parallel/ParallelIndexSupervisorTask.java | 109 +++-- .../overlord/LockRequestForNewSegment.java | 22 +- .../druid/indexing/overlord/TaskLockbox.java | 33 +- .../overlord/TimeChunkLockRequest.java | 13 + .../SeekableStreamIndexTask.java | 8 +- .../SeekableStreamIndexTaskRunner.java | 38 +- .../actions/SegmentAllocateActionTest.java | 28 +- .../common/task/CompactionTaskRunTest.java | 121 ++++- .../indexing/common/task/HadoopTaskTest.java | 12 +- .../indexing/common/task/IndexTaskTest.java | 53 +- .../ParallelIndexSupervisorTaskTest.java | 74 ++- .../indexing/overlord/TaskLockboxTest.java | 1 + pom.xml | 2 +- .../IndexerMetadataStorageCoordinator.java | 2 + .../IndexerSQLMetadataStorageCoordinator.java | 4 + ... AppenderatorDriverSegmentLockHelper.java} | 4 +- .../appenderator/BaseAppenderatorDriver.java | 2 +- .../appenderator/BatchAppenderatorDriver.java | 8 +- .../StreamAppenderatorDriver.java | 2 +- 58 files changed, 1378 insertions(+), 834 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/actions/{SegmentLockAquireAction.java => SegmentLockAcquireAction.java} (89%) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java rename server/src/main/java/org/apache/druid/segment/realtime/appenderator/{StreamAppenderatorDriverSegmentLockHelper.java => AppenderatorDriverSegmentLockHelper.java} (89%) 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 4fdbca71fd45..01ed140376b8 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 @@ -63,10 +63,10 @@ public DateTime toDate(String filePath, Formatter formatter) } /** - * All granularity matches no interval. + * No interval is aligned with all granularity since it's infinite. */ @Override - public boolean match(Interval interval) + public boolean isAligned(Interval interval) { return false; } 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 78748836930b..d9af7eda819d 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 @@ -96,7 +96,7 @@ public DateTime toDate(String filePath, Formatter formatter) } @Override - public boolean match(Interval interval) + public boolean isAligned(Interval interval) { if (interval.toDurationMillis() == duration) { return (interval.getStartMillis() - origin) % duration == 0; 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 ad9ccd8a8b92..3a9cdac17b73 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 @@ -116,7 +116,7 @@ public static List granularitiesFinerThan(final Granularity gran0) /** * Return true if time chunks populated by this granularity includes the given interval time chunk. */ - public abstract boolean match(Interval interval); + public abstract boolean isAligned(Interval interval); public DateTime bucketEnd(DateTime 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 b393d814c594..750ea2a335e1 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 @@ -61,10 +61,10 @@ public DateTime toDate(String filePath, Formatter formatter) } /** - * None granularity matches to any interval. + * Any interval is aligned with none granularity since it's effectively millisecond granularity. */ @Override - public boolean match(Interval interval) + public boolean isAligned(Interval interval) { return true; } 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 13f5e12b7def..b1f36e6c6724 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 @@ -138,7 +138,7 @@ public DateTime toDate(String filePath, Formatter formatter) } @Override - public boolean match(Interval interval) + public boolean isAligned(Interval interval) { return bucket(interval.getStart()).equals(interval); } diff --git a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java index 90cd8f756c63..562e8661a8e9 100644 --- a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java +++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java @@ -65,16 +65,6 @@ default boolean containsRootPartition(T other) */ int getEndRootPartitionId(); - default short getStartRootPartitionIdAsShort() - { - return (short) getStartRootPartitionId(); - } - - default short getEndRootPartitionIdAsShort() - { - return (short) getEndRootPartitionId(); - } - String getMajorVersion(); short getMinorVersion(); 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 e3f967b458b3..761ec5e6397a 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -184,9 +184,8 @@ private void addAll( public PartitionChunk remove(Interval interval, VersionType version, PartitionChunk chunk) { + lock.writeLock().lock(); try { - lock.writeLock().lock(); - Map versionEntries = allTimelineEntries.get(interval); if (versionEntries == null) { return null; @@ -219,8 +218,8 @@ public PartitionChunk remove(Interval interval, VersionType version, @Override public PartitionHolder findEntry(Interval interval, VersionType version) { + lock.readLock().lock(); try { - lock.readLock().lock(); for (Entry> entry : allTimelineEntries.entrySet()) { if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) { TimelineEntry foundEntry = entry.getValue().get(version); @@ -251,8 +250,8 @@ public PartitionHolder findEntry(Interval interval, VersionType vers @Override public List> lookup(Interval interval) { + lock.readLock().lock(); try { - lock.readLock().lock(); return lookup(interval, false); } finally { @@ -263,8 +262,8 @@ public List> lookup(Interval inter @Override public List> lookupWithIncompletePartitions(Interval interval) { + lock.readLock().lock(); try { - lock.readLock().lock(); return lookup(interval, true); } finally { @@ -274,8 +273,8 @@ public List> lookupWithIncompleteP public boolean isEmpty() { + lock.readLock().lock(); try { - lock.readLock().lock(); return completePartitionsTimeline.isEmpty(); } finally { @@ -285,8 +284,8 @@ public boolean isEmpty() public TimelineObjectHolder first() { + lock.readLock().lock(); try { - lock.readLock().lock(); return timelineEntryToObjectHolder(completePartitionsTimeline.firstEntry().getValue()); } finally { @@ -296,8 +295,8 @@ public TimelineObjectHolder first() public TimelineObjectHolder last() { + lock.readLock().lock(); try { - lock.readLock().lock(); return timelineEntryToObjectHolder(completePartitionsTimeline.lastEntry().getValue()); } finally { @@ -376,9 +375,8 @@ public Set> findFullyOvershadowed( public boolean isOvershadowed(Interval interval, VersionType version, ObjectType object) { + lock.readLock().lock(); try { - lock.readLock().lock(); - TimelineEntry entry = completePartitionsTimeline.get(interval); if (entry != null) { final int majorVersionCompare = versionComparator.compare(version, entry.getVersion()); @@ -493,13 +491,14 @@ private boolean addAtKey( ); if (versionCompare < 0) { - // since the entry version is lower than the existing one, the existing one overwrites the given entry if overlapped. + // 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())) { - // | cur | // | entry | + // | cur | // => |new| entryInterval = new Interval(currKey.getEnd(), entryInterval.getEnd()); } else { @@ -514,11 +513,14 @@ private boolean addAtKey( 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) { - // since the entry version is greater than the existing one, the given entry overwrites the existing one if overlapped. + // 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)) { 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 index 0216bacbb779..b200a4d896cd 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -79,7 +79,7 @@ public void remove(PartitionChunk chunk) public boolean isFull() { - return chunks.size() == chunks.get(0).getObject().getAtomicUpdateGroupSize(); + return !isEmpty() && chunks.size() == chunks.get(0).getObject().getAtomicUpdateGroupSize(); } public boolean isEmpty() @@ -118,13 +118,6 @@ public int getStartRootPartitionId() return chunks.get(0).getObject().getStartRootPartitionId(); } - @Override - public short getStartRootPartitionIdAsShort() - { - Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); - return chunks.get(0).getObject().getStartRootPartitionIdAsShort(); - } - @Override public int getEndRootPartitionId() { @@ -132,13 +125,6 @@ public int getEndRootPartitionId() return chunks.get(0).getObject().getEndRootPartitionId(); } - @Override - public short getEndRootPartitionIdAsShort() - { - Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); - return chunks.get(0).getObject().getEndRootPartitionIdAsShort(); - } - @Override public String getMajorVersion() { 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 index 2901ed5d1a32..764917c9346b 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwritingShardSpecFactory.java @@ -64,6 +64,8 @@ public short getMinorVersion() @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 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 index 3211dea6e5a9..72f6a38ec5e2 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -126,11 +126,15 @@ private void transitPartitionChunkState(AtomicUpdateGroup atomicUpdateGroup, Preconditions.checkArgument(!atomicUpdateGroup.isEmpty(), "empty atomicUpdateGroup"); removeFrom(atomicUpdateGroup, from); - addTo(atomicUpdateGroup, to); + 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 searchForStateOf(PartitionChunk chunk, State state) + private AtomicUpdateGroup findAtomicUpdateGroupWith(PartitionChunk chunk, State state) { final Short2ObjectSortedMap> versionToGroup = getStateMap(state).get( RootPartitionRange.of(chunk) @@ -149,7 +153,7 @@ private AtomicUpdateGroup searchForStateOf(PartitionChunk chunk, State sta * Can return an empty atomicUpdateGroup. */ @Nullable - private AtomicUpdateGroup tryRemoveFromState(PartitionChunk chunk, State state) + private AtomicUpdateGroup tryRemoveChunkFromGroupWithState(PartitionChunk chunk, State state) { final RootPartitionRange rangeKey = RootPartitionRange.of(chunk); final Short2ObjectSortedMap> versionToGroup = getStateMap(state).get(rangeKey); @@ -164,7 +168,7 @@ private AtomicUpdateGroup tryRemoveFromState(PartitionChunk chunk, State s } } - handleRemove(atomicUpdateGroup, RootPartitionRange.of(chunk), chunk.getObject().getMinorVersion(), state); + determineVisibleGroupAfterRemove(atomicUpdateGroup, RootPartitionRange.of(chunk), chunk.getObject().getMinorVersion(), state); return atomicUpdateGroup; } } @@ -216,7 +220,7 @@ private List>> findOvershadowedBy( /** * Handles addition of the atomicUpdateGroup to the given state */ - private void handleAdd(AtomicUpdateGroup aug, State newStateOfAug) + private void transitionStandbyGroupIfFull(AtomicUpdateGroup aug, State newStateOfAug) { if (newStateOfAug == State.STANDBY) { // A standby atomicUpdateGroup becomes visible when its all segments are available. @@ -232,7 +236,7 @@ private void handleAdd(AtomicUpdateGroup aug, State newStateOfAug) } } - private void addTo(AtomicUpdateGroup aug, State state) + private void addAtomicUpdateGroupWithState(AtomicUpdateGroup aug, State state) { final AtomicUpdateGroup existing = getStateMap(state) .computeIfAbsent(RootPartitionRange.of(aug), k -> createMinorVersionToAugMap(state)) @@ -242,11 +246,12 @@ private void addTo(AtomicUpdateGroup aug, State state) throw new ISE("AtomicUpdateGroup[%s] is already in state[%s]", aug, state); } - handleAdd(aug, state); + transitionStandbyGroupIfFull(aug, state); } - public void add(PartitionChunk chunk) + public void addChunk(PartitionChunk chunk) { + // Sanity check. ExistingChunk should be usually null. final PartitionChunk existingChunk = knownPartitionChunks.put(chunk.getChunkNumber(), chunk); if (existingChunk != null && !existingChunk.equals(chunk)) { throw new ISE( @@ -258,18 +263,18 @@ public void add(PartitionChunk chunk) } // Find atomicUpdateGroup of the new chunk - AtomicUpdateGroup atomicUpdateGroup = searchForStateOf(chunk, State.OVERSHADOWED); + AtomicUpdateGroup atomicUpdateGroup = findAtomicUpdateGroupWith(chunk, State.OVERSHADOWED); if (atomicUpdateGroup != null) { atomicUpdateGroup.add(chunk); } else { - atomicUpdateGroup = searchForStateOf(chunk, State.STANDBY); + atomicUpdateGroup = findAtomicUpdateGroupWith(chunk, State.STANDBY); if (atomicUpdateGroup != null) { atomicUpdateGroup.add(chunk); - handleAdd(atomicUpdateGroup, State.STANDBY); + transitionStandbyGroupIfFull(atomicUpdateGroup, State.STANDBY); } else { - atomicUpdateGroup = searchForStateOf(chunk, State.VISIBLE); + atomicUpdateGroup = findAtomicUpdateGroupWith(chunk, State.VISIBLE); if (atomicUpdateGroup != null) { // A new chunk of the same major version and partitionId can be added in segment handoff @@ -299,9 +304,9 @@ public void add(PartitionChunk chunk) .anyMatch(group -> group.isOvershadow(newAtomicUpdateGroup)); if (overshadowed) { - addTo(newAtomicUpdateGroup, State.OVERSHADOWED); + addAtomicUpdateGroupWithState(newAtomicUpdateGroup, State.OVERSHADOWED); } else { - addTo(newAtomicUpdateGroup, State.STANDBY); + addAtomicUpdateGroupWithState(newAtomicUpdateGroup, State.STANDBY); } } } @@ -311,19 +316,19 @@ public void add(PartitionChunk chunk) /** * Handles of removal of an empty atomicUpdateGroup from a state. */ - private void handleRemove( + private void determineVisibleGroupAfterRemove( AtomicUpdateGroup augOfRemovedChunk, RootPartitionRange rangeOfAug, short minorVersion, State stateOfRemovedAug ) { - if (stateOfRemovedAug == State.STANDBY) { - // If an atomicUpdateGroup is overshadowed by another standby atomicUpdateGroup, there must be another visible - // atomicUpdateGroup which also overshadows the same atomicUpdateGroup. - // As a result, the state of overshadowed atomicUpdateGroup shouldn't be changed and we do nothing here. + // 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. - } else if (stateOfRemovedAug == State.VISIBLE) { + 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 @@ -332,6 +337,8 @@ private void handleRemove( 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 @@ -340,8 +347,6 @@ private void handleRemove( } latestFullAugs.forEach(group -> transitPartitionChunkState(group, State.OVERSHADOWED, State.VISIBLE)); } - } else { - // do nothing } } @@ -362,7 +367,7 @@ private List> findLatestFullyAvailableOvershadowedAtomicUpd final OvershadowableManager manager = new OvershadowableManager<>(); overshadowedGroups.stream() .flatMap(entry -> entry.getValue().getChunks().stream()) - .forEach(manager::add); + .forEach(manager::addChunk); return manager.visibleGroup .values() @@ -398,7 +403,7 @@ private void removeFrom(AtomicUpdateGroup aug, State state) } @Nullable - public PartitionChunk remove(PartitionChunk partitionChunk) + public PartitionChunk removeChunk(PartitionChunk partitionChunk) { final PartitionChunk knownChunk = knownPartitionChunks.get(partitionChunk.getChunkNumber()); if (knownChunk == null) { @@ -414,12 +419,12 @@ public PartitionChunk remove(PartitionChunk partitionChunk) ); } - AtomicUpdateGroup augOfRemovedChunk = tryRemoveFromState(partitionChunk, State.STANDBY); + AtomicUpdateGroup augOfRemovedChunk = tryRemoveChunkFromGroupWithState(partitionChunk, State.STANDBY); if (augOfRemovedChunk == null) { - augOfRemovedChunk = tryRemoveFromState(partitionChunk, State.VISIBLE); + augOfRemovedChunk = tryRemoveChunkFromGroupWithState(partitionChunk, State.VISIBLE); if (augOfRemovedChunk == null) { - augOfRemovedChunk = tryRemoveFromState(partitionChunk, State.OVERSHADOWED); + augOfRemovedChunk = tryRemoveChunkFromGroupWithState(partitionChunk, State.OVERSHADOWED); if (augOfRemovedChunk == null) { throw new ISE("Can't find atomicUpdateGroup for partitionChunk[%s]", partitionChunk); } @@ -446,7 +451,7 @@ public PartitionChunk getChunk(int partitionId) if (chunk == null) { return null; } - final AtomicUpdateGroup aug = searchForStateOf(chunk, State.VISIBLE); + final AtomicUpdateGroup aug = findAtomicUpdateGroupWith(chunk, State.VISIBLE); if (aug == null) { return null; } else { 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 4c70061e1b16..ed2761014340 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 @@ -58,12 +58,12 @@ public PartitionHolder(PartitionHolder partitionHolder) public void add(PartitionChunk chunk) { - overshadowableManager.add(chunk); + overshadowableManager.addChunk(chunk); } public PartitionChunk remove(PartitionChunk chunk) { - return overshadowableManager.remove(chunk); + return overshadowableManager.removeChunk(chunk); } public boolean isEmpty() 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 index 5b2c922e4e3a..7ab0be836d59 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecFactory.java @@ -39,6 +39,8 @@ 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); 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 bf0580cd7ad0..dd0fa7269556 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.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 IncrementalPublishingKafkaIndexTaskRunner( AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, CircularBuffer savedParseExceptions, - RowIngestionMetersFactory rowIngestionMetersFactory + RowIngestionMetersFactory rowIngestionMetersFactory, + LockGranularity lockGranularityToUse ) { super( @@ -79,7 +81,8 @@ public IncrementalPublishingKafkaIndexTaskRunner( 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 b0b0c6be7470..25140d8d12f6 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 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() 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 fe1e01fa894b..39e5342651bf 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 @@ -50,6 +50,7 @@ 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 +68,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.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; @@ -163,6 +165,8 @@ 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; @@ -189,6 +193,7 @@ import static org.apache.druid.query.QueryPlus.wrap; +@RunWith(Parameterized.class) public class KafkaIndexTaskTest { private static final Logger log = new Logger(KafkaIndexTaskTest.class); @@ -205,7 +210,17 @@ public class KafkaIndexTaskTest new KafkaIndexTaskModule().getJacksonModules().forEach(OBJECT_MAPPER::registerModule); } + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK}, + new Object[]{LockGranularity.SEGMENT} + ); + } + private final List runningTasks = new ArrayList<>(); + private final LockGranularity lockGranularity; private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; @@ -314,6 +329,11 @@ private static String getTopicName() @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); + public KafkaIndexTaskTest(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + } + @BeforeClass public static void setupClass() throws Exception { @@ -2422,6 +2442,7 @@ private ListenableFuture runTask(final Task task) 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 { 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 f3dfe3bfcef0..3490ee99050e 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 KinesisIndexTask( this.awsCredentialsConfig = awsCredentialsConfig; } - @Override protected SeekableStreamIndexTaskRunner createTaskRunner() { @@ -79,7 +78,8 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() 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 335119a08b19..3e88bfdb5787 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.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 chatHandlerProvider, CircularBuffer savedParseExceptions, - RowIngestionMetersFactory rowIngestionMetersFactory + RowIngestionMetersFactory rowIngestionMetersFactory, + LockGranularity lockGranularityToUse ) { super( @@ -73,7 +75,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner runningTasks = new ArrayList<>(); + @Parameterized.Parameters(name = "{0}") + public static Iterable 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; @@ -291,6 +307,11 @@ public static void setupClass() ); } + public KinesisIndexTaskTest(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + } + @Before public void setupTest() throws IOException, InterruptedException { @@ -2591,6 +2612,7 @@ private ListenableFuture runTask(final Task task) 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 { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java index 07d3f30d4c5a..80a0344d8446 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java @@ -24,6 +24,8 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import java.io.IOException; + /** * This class is used in {@link ActionBasedSegmentAllocator} and expected to generate a * {@link TaskAction< SegmentIdWithShardSpec >} which is submitted to overlords to allocate a new segment. @@ -40,5 +42,5 @@ TaskAction generate( String sequenceName, String previousSegmentId, boolean skipSegmentLineageCheck - ); + ) throws IOException; } 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 13cb7b19a245..aecfcec36432 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 @@ -24,6 +24,7 @@ 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.TaskLockType; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -75,6 +76,7 @@ public class SegmentAllocateAction implements TaskAction private final String previousSegmentId; private final boolean skipSegmentLineageCheck; private final ShardSpecFactory shardSpecFactory; + private final LockGranularity lockGranularity; @JsonCreator public SegmentAllocateAction( @@ -85,7 +87,8 @@ public SegmentAllocateAction( @JsonProperty("sequenceName") String sequenceName, @JsonProperty("previousSegmentId") String previousSegmentId, @JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck, - @JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory + @JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory, // nullable for backward compatibility + @JsonProperty("lockGranularity") @Nullable LockGranularity lockGranularity // nullable for backward compatibility ) { this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); @@ -99,6 +102,7 @@ public SegmentAllocateAction( this.previousSegmentId = previousSegmentId; this.skipSegmentLineageCheck = skipSegmentLineageCheck; this.shardSpecFactory = shardSpecFactory == null ? NumberedShardSpecFactory.instance() : shardSpecFactory; + this.lockGranularity = lockGranularity == null ? LockGranularity.TIME_CHUNK : lockGranularity; } @JsonProperty @@ -149,6 +153,12 @@ public ShardSpecFactory getShardSpecFactory() return shardSpecFactory; } + @JsonProperty + public LockGranularity getLockGranularity() + { + return lockGranularity; + } + @Override public TypeReference getReturnTypeReference() { @@ -278,6 +288,7 @@ private SegmentIdWithShardSpec tryAllocate( final LockResult lockResult = toolbox.getTaskLockbox().tryLock( task, new LockRequestForNewSegment( + lockGranularity, TaskLockType.EXCLUSIVE, task.getGroupId(), dataSource, @@ -345,6 +356,7 @@ public String toString() ", previousSegmentId='" + previousSegmentId + '\'' + ", skipSegmentLineageCheck=" + skipSegmentLineageCheck + ", shardSpecFactory=" + shardSpecFactory + + ", lockGranularity=" + lockGranularity + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java similarity index 89% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java index 76bf8c73ebce..70c81225b2f5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockAcquireAction.java @@ -30,9 +30,13 @@ import org.joda.time.Interval; /** - * TaskAction to acquire a task lock for a segment. Used by stream ingestion tasks. + * 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 SegmentLockAquireAction implements TaskAction +public class SegmentLockAcquireAction implements TaskAction { private final TaskLockType lockType; private final Interval interval; @@ -41,7 +45,7 @@ public class SegmentLockAquireAction implements TaskAction private final long timeoutMs; @JsonCreator - public SegmentLockAquireAction( + public SegmentLockAcquireAction( @JsonProperty("lockType") TaskLockType lockType, @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @@ -124,7 +128,7 @@ public boolean isAudited() @Override public String toString() { - return "SegmentLockAquireAction{" + + return "SegmentLockAcquireAction{" + "lockType=" + lockType + ", interval=" + interval + ", version='" + version + '\'' + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java index 96f5e391c858..7728574c0756 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentLockTryAcquireAction.java @@ -35,6 +35,7 @@ /** * 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> { 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 b585a80f326e..e1428c6c9618 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 @@ -26,8 +26,8 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; 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; @@ -201,8 +201,8 @@ private void checkWithSegmentLock() final Map> oldSegmentsMap = groupSegmentsByIntervalAndSort(segmentsToBeOverwritten); final Map> newSegmentsMap = groupSegmentsByIntervalAndSort(segments); - oldSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); - newSegmentsMap.values().forEach(AbstractBatchIndexTask::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); + oldSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); + newSegmentsMap.values().forEach(SegmentLockHelper::verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull); oldSegmentsMap.forEach((interval, oldSegmentsPerInterval) -> { final List newSegmentsPerInterval = Preconditions.checkNotNull( 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 392ebbd24af0..e268188a464e 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 @@ -29,7 +29,7 @@ @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 = SegmentLockAquireAction.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/TimeChunkLockAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java index 1441fc1b6c70..83eec1351522 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockAcquireAction.java @@ -33,6 +33,11 @@ import javax.annotation.Nullable; +/** + * 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 { private final TaskLockType type; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java index e4c54d5050b9..7c2b7e0bf98c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TimeChunkLockTryAcquireAction.java @@ -32,6 +32,10 @@ import javax.annotation.Nullable; +/** + * 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 { @JsonIgnore 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 index 787636effc74..166549e975c5 100644 --- 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 @@ -21,84 +21,54 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import org.apache.druid.indexing.common.SegmentLock; +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.SegmentLockReleaseAction; -import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; +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.overlord.LockResult; -import org.apache.druid.java.util.common.ISE; +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.io.Closer; +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.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; 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 ({@link #tryLockWithIntervals} and {@link #tryLockWithSegments}) for easily acquiring task + * Provides some methods ({@link #determineSegmentGranularity} and {@link #determineSegmentGranularity}) for easily acquiring task * locks. */ public abstract class AbstractBatchIndexTask extends AbstractTask { - @Nullable - private Map overwritingRootGenPartitions; - @Nullable - private Set allInputSegments; - - @Nullable - private Boolean changeSegmentGranularity; + private static final Logger log = new Logger(AbstractBatchIndexTask.class); - public static class OverwritingRootGenerationPartitions - { - private final int startRootPartitionId; - private final int endRootPartitionId; - private final short maxMinorVersion; - - private OverwritingRootGenerationPartitions(int startRootPartitionId, int endRootPartitionId, short maxMinorVersion) - { - this.startRootPartitionId = startRootPartitionId; - this.endRootPartitionId = endRootPartitionId; - this.maxMinorVersion = maxMinorVersion; - } - - public int getStartRootPartitionId() - { - return startRootPartitionId; - } - - public int getEndRootPartitionId() - { - return endRootPartitionId; - } - - public short getMinorVersionForNewSegments() - { - return (short) (maxMinorVersion + 1); - } - } + private final SegmentLockHelper segmentLockHelper; + private boolean useSegmentLock; protected AbstractBatchIndexTask(String id, String dataSource, Map context) { super(id, dataSource, context); + segmentLockHelper = new SegmentLockHelper(dataSource); } protected AbstractBatchIndexTask( @@ -110,60 +80,143 @@ protected AbstractBatchIndexTask( ) { super(id, groupId, taskResource, dataSource, context); + segmentLockHelper = new SegmentLockHelper(dataSource); } - public abstract boolean requireLockInputSegments(); + /** + * Return true if this task can overwrite existing segments. + */ + public abstract boolean requireLockExistingSegments(); - public abstract List findInputSegments(TaskActionClient taskActionClient, List intervals) + /** + * 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. + */ + // TODO: remove this + public abstract List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException; - public abstract boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments); - public abstract boolean isPerfectRollup(); - /** - * Returns the segmentGranularity for the given interval. Usually tasks are supposed to return its segmentGranularity - * if exists. The compactionTask can return different segmentGranularity depending on its configuration and the input - * interval. - * - * @return segmentGranularity or null if it doesn't support it. - */ + public boolean isUseSegmentLock() + { + return useSegmentLock; + } + @Nullable - public abstract Granularity getSegmentGranularity(Interval interval); + public abstract Granularity getSegmentGranularity(); - protected boolean tryLockWithIntervals(TaskActionClient client, List intervals, boolean isInitialRequest) + public boolean determineLockGranularityAndTryLock( + TaskActionClient client, + GranularitySpec granularitySpec + ) throws IOException + { + final List intervals = granularitySpec.bucketIntervals().isPresent() + ? new ArrayList<>(granularitySpec.bucketIntervals().get()) + : Collections.emptyList(); + return determineLockGranularityandTryLock(client, intervals); + } + + public SegmentLockHelper getNonNullSegmentLockHelper() + { + return Preconditions.checkNotNull(segmentLockHelper, "segmentLockHelper"); + } + + protected boolean determineLockGranularityandTryLock(TaskActionClient client, List intervals) throws IOException { - if (requireLockInputSegments()) { - if (isPerfectRollup()) { + 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; + if (!intervals.isEmpty()) { return tryTimeChunkLock(client, intervals); + } else { + return true; + } + } else { + if (!intervals.isEmpty()) { + final LockGranularityDeterminResult result = determineSegmentGranularity(client, intervals); + useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT; + return tryLockWithDetermineResult(client, result); + } else { + return true; + } + } + } + + protected boolean determineLockGranularityandTryLockWithSegments(TaskActionClient client, List 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 LockGranularityDeterminResult result = determineSegmentGranularity(segments); + useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT; + return tryLockWithDetermineResult(client, result); + } + } + + private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranularityDeterminResult result) + throws IOException + { + if (result.lockGranularity == LockGranularity.TIME_CHUNK) { + return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals")); + } else { + final boolean isReady = segmentLockHelper.verifyAndLockExistingSegments( + client, + Preconditions.checkNotNull(result.segments, "segments") + ); + return isReady; + } + } + + protected LockGranularityDeterminResult determineSegmentGranularity( + TaskActionClient client, + List intervals + ) throws IOException + { + if (requireLockExistingSegments()) { + if (isPerfectRollup()) { + log.info("Using timeChunk lock for perfrect rollup"); + return new LockGranularityDeterminResult(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 findInputSegments() and tryLockWithSegments(), + // 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 findInputSegments() call, it may missing those + // 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 tryLockWithSegments(client, findInputSegments(client, intervals), isInitialRequest); + return determineSegmentGranularity(findSegmentsToLock(client, intervals)); } else { - return true; + log.info("Using segment lock for empty intervals"); + return new LockGranularityDeterminResult(LockGranularity.SEGMENT, null, Collections.emptyList()); } } else { - changeSegmentGranularity = false; - allInputSegments = Collections.emptySet(); - overwritingRootGenPartitions = Collections.emptyMap(); - return true; + log.info("Using segment lock since we don't have to lock existing segments"); + return new LockGranularityDeterminResult(LockGranularity.SEGMENT, null, Collections.emptyList()); } } private boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException { - allInputSegments = Collections.emptySet(); - overwritingRootGenPartitions = Collections.emptyMap(); // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined final Set uniqueIntervals = new HashSet<>(); for (Interval interval : JodaUtils.condenseIntervals(intervals)) { - final Granularity segmentGranularity = getSegmentGranularity(interval); + final Granularity segmentGranularity = getSegmentGranularity(); if (segmentGranularity == null) { uniqueIntervals.add(interval); } else { @@ -180,198 +233,130 @@ private boolean tryTimeChunkLock(TaskActionClient client, List interva return true; } - boolean tryLockWithSegments(TaskActionClient client, List segments, boolean isInitialRequest) - throws IOException + @Nullable + public static Granularity findGranularityFromSegments(List 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; + } + } + + LockGranularityDeterminResult determineSegmentGranularity(List segments) { if (segments.isEmpty()) { - changeSegmentGranularity = false; - allInputSegments = Collections.emptySet(); - overwritingRootGenPartitions = Collections.emptyMap(); - return true; + 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 LockGranularityDeterminResult(LockGranularity.SEGMENT, null, Collections.emptyList()); } - if (requireLockInputSegments()) { - // Create a timeline to find latest segments only + if (requireLockExistingSegments()) { + final Granularity granularityFromSegments = findGranularityFromSegments(segments); + @Nullable + final Granularity segmentGranularityFromSpec = getSegmentGranularity(); final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); - changeSegmentGranularity = checkIfChangeSegmentGranularity(intervals); - if (changeSegmentGranularity) { - return tryTimeChunkLock(client, intervals); + 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 LockGranularityDeterminResult(LockGranularity.TIME_CHUNK, intervals, null); } else { - final List segmentsToLock; + // Use segment lock + // Create a timeline to find latest segments only final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( segments ); - segmentsToLock = timeline.lookup(JodaUtils.umbrellaInterval(intervals)) - .stream() - .map(TimelineObjectHolder::getObject) - .flatMap(partitionHolder -> StreamSupport.stream( - partitionHolder.spliterator(), - false - )) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); - if (allInputSegments == null) { - allInputSegments = new HashSet<>(segmentsToLock); - overwritingRootGenPartitions = new HashMap<>(); - } - - final Map> intervalToSegments = new HashMap<>(); - for (DataSegment segment : segmentsToLock) { - intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); - } - intervalToSegments.values().forEach( - segmentsToCheck -> verifyAndFindRootPartitionRangeAndMinorVersion(segmentsToCheck, isInitialRequest) - ); - final Closer lockCloserOnError = Closer.create(); - for (Entry> entry : intervalToSegments.entrySet()) { - final Interval interval = entry.getKey(); - final Set partitionIds = entry.getValue().stream() - .map(s -> s.getShardSpec().getPartitionNum()) - .collect(Collectors.toSet()); - final List lockResults = client.submit( - new SegmentLockTryAcquireAction( - TaskLockType.EXCLUSIVE, - interval, - entry.getValue().get(0).getMajorVersion(), - partitionIds - ) - ); - - lockResults.stream() - .filter(LockResult::isOk) - .map(result -> (SegmentLock) result.getTaskLock()) - .forEach(segmentLock -> lockCloserOnError.register(() -> client.submit( - new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) - ))); - - if (isInitialRequest && (lockResults.isEmpty() || lockResults.stream().anyMatch(result -> !result.isOk()))) { - lockCloserOnError.close(); - return false; - } - } - return true; + final List 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 LockGranularityDeterminResult(LockGranularity.SEGMENT, null, segmentsToLock); } } else { - changeSegmentGranularity = false; - allInputSegments = Collections.emptySet(); - overwritingRootGenPartitions = Collections.emptyMap(); - return true; + // 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 LockGranularityDeterminResult(LockGranularity.SEGMENT, null, Collections.emptyList()); } } - /** - * This method is called when the task overwrites existing segments with segment locks. It verifies the input segments - * can be locked together, so that output segments can overshadow existing ones properly. - *

- * This method checks two things: - *

- * - Are rootPartition range of inputSegments adjacent? Two rootPartition ranges are adjacent if they are consecutive. - * - All atomicUpdateGroups of inputSegments must be full. (See {@code AtomicUpdateGroup#isFull()}). - */ - private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments, boolean isInitialRequest) + protected static List findInputSegments( + String dataSource, + TaskActionClient actionClient, + List intervalsToFind, // TODO: must be checked somewhere? probably? + FirehoseFactory firehoseFactory + ) throws IOException { - if (inputSegments.isEmpty()) { - return; - } + if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { + final List inputSegments = ((IngestSegmentFirehoseFactory) firehoseFactory).getSegments(); + if (inputSegments == null) { + final Interval inputInterval = Preconditions.checkNotNull( + ((IngestSegmentFirehoseFactory) firehoseFactory).getInterval(), + "input interval" + ); - final List sortedSegments = new ArrayList<>(inputSegments); - sortedSegments.sort((s1, s2) -> { - if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) { - return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId()); + return actionClient.submit( + new SegmentListUsedAction(dataSource, null, Collections.singletonList(inputInterval)) + ); } else { - return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId()); + final List inputSegmentIds = inputSegments.stream() + .map(WindowedSegmentId::getSegmentId) + .collect(Collectors.toList()); + final List 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()); } - }); - if (isInitialRequest) { - verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(sortedSegments); + } else { + return actionClient.submit(new SegmentListUsedAction(dataSource, null, intervalsToFind)); } - final Interval interval = sortedSegments.get(0).getInterval(); - final short prevMaxMinorVersion = (short) sortedSegments - .stream() - .mapToInt(DataSegment::getMinorVersion) - .max() - .orElseThrow(() -> new ISE("Empty inputSegments")); - - overwritingRootGenPartitions.put( - interval, - new OverwritingRootGenerationPartitions( - sortedSegments.get(0).getStartRootPartitionId(), - sortedSegments.get(sortedSegments.size() - 1).getEndRootPartitionId(), - prevMaxMinorVersion - ) - ); - } - - public Set getAllInputSegments() - { - return Preconditions.checkNotNull(allInputSegments, "allInputSegments is not initialized"); - } - - Map getAllOverwritingSegmentMeta() - { - Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); - return Collections.unmodifiableMap(overwritingRootGenPartitions); - } - - public boolean isChangeSegmentGranularity() - { - return Preconditions.checkNotNull(changeSegmentGranularity, "changeSegmentGranularity is not initialized"); - } - - public boolean hasInputSegments() - { - Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); - return !overwritingRootGenPartitions.isEmpty(); } - @Nullable - public OverwritingRootGenerationPartitions getOverwritingSegmentMeta(Interval interval) + private static class LockGranularityDeterminResult { - Preconditions.checkNotNull(overwritingRootGenPartitions, "overwritingRootGenPartitions is not initialized"); - return overwritingRootGenPartitions.get(interval); - } - - public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List sortedSegments) - { - if (sortedSegments.isEmpty()) { - return; - } - - Preconditions.checkArgument( - sortedSegments.stream().allMatch(segment -> segment.getInterval().equals(sortedSegments.get(0).getInterval())) - ); - - short atomicUpdateGroupSize = 1; - // sanity check - for (int i = 0; i < sortedSegments.size() - 1; i++) { - final DataSegment curSegment = sortedSegments.get(i); - final DataSegment nextSegment = sortedSegments.get(i + 1); - if (curSegment.getStartRootPartitionId() == nextSegment.getStartRootPartitionId() - && curSegment.getEndRootPartitionId() == nextSegment.getEndRootPartitionId()) { - // Input segments should have the same or consecutive rootPartition range - if (curSegment.getMinorVersion() != nextSegment.getMinorVersion() - || curSegment.getAtomicUpdateGroupSize() != nextSegment.getAtomicUpdateGroupSize()) { - throw new ISE( - "segment[%s] and segment[%s] have the same rootPartitionRange, but different minorVersion or atomicUpdateGroupSize", - curSegment, - nextSegment - ); - } - atomicUpdateGroupSize++; - } else { - if (curSegment.getEndRootPartitionId() != nextSegment.getStartRootPartitionId()) { - throw new ISE("Can't compact segments of non-consecutive rootPartition range"); - } - if (atomicUpdateGroupSize != curSegment.getAtomicUpdateGroupSize()) { - throw new ISE("All atomicUpdateGroup must be compacted together"); - } - atomicUpdateGroupSize = 1; - } - } - if (atomicUpdateGroupSize != sortedSegments.get(sortedSegments.size() - 1).getAtomicUpdateGroupSize()) { - throw new ISE("All atomicUpdateGroup must be compacted together"); + private final LockGranularity lockGranularity; + @Nullable + private final List intervals; + @Nullable + private final List segments; + + private LockGranularityDeterminResult( + LockGranularity lockGranularity, + @Nullable List intervals, + @Nullable List segments + ) + { + this.lockGranularity = lockGranularity; + this.intervals = intervals; + this.segments = segments; } } } 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 8e6834ffb0be..9c51fa9ad6fa 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,13 @@ 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.SegmentLockAquireAction; +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.config.TaskConfig; @@ -282,7 +283,7 @@ public TaskStatus run(final TaskToolbox toolbox) segmentId -> { try { return toolbox.getTaskActionClient().submit( - new SegmentLockAquireAction( + new SegmentLockAcquireAction( TaskLockType.EXCLUSIVE, segmentId.getInterval(), segmentId.getVersion(), @@ -755,7 +756,8 @@ private static StreamAppenderatorDriver newDriver( sequenceName, previousSegmentId, skipSegmentLineageCheck, - NumberedShardSpecFactory.instance() + 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 index 707372458491..0fa6f8791b32 100644 --- 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 @@ -20,16 +20,22 @@ 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; @@ -38,10 +44,20 @@ import java.util.stream.IntStream; /** - * CachingSegmentAllocator which allocates segments locally. + * Allocates all necessary segments at the beginning and reuse them. */ -class CachingLocalSegmentAllocator extends CachingSegmentAllocator +class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator { + private final TaskToolbox toolbox; + private final String taskId; + private final String dataSource; + private final Map> allocateSpec; + @Nullable + private final ShardSpecs shardSpecs; + + // sequenceName -> segmentId + private final Map sequenceNameToSegmentId; + CachingLocalSegmentAllocator( TaskToolbox toolbox, String taskId, @@ -49,13 +65,29 @@ class CachingLocalSegmentAllocator extends CachingSegmentAllocator Map> allocateSpec ) throws IOException { - // This segment allocator doesn't need inputPartitionIds because the newly created segments don't have to store - // direcOvershadowingSegments - super(toolbox, taskId, dataSource, allocateSpec); + this.toolbox = toolbox; + this.taskId = taskId; + this.dataSource = dataSource; + this.allocateSpec = allocateSpec; + this.sequenceNameToSegmentId = new HashMap<>(); + + final Map> intervalToIds = getIntervalToSegmentIds(); + final Map> shardSpecMap = new HashMap<>(); + + for (Map.Entry> entry : intervalToIds.entrySet()) { + final Interval interval = entry.getKey(); + final List idsPerInterval = intervalToIds.get(interval); + + for (SegmentIdWithShardSpec segmentIdentifier : idsPerInterval) { + shardSpecMap.computeIfAbsent(interval, k -> new ArrayList<>()).add(segmentIdentifier.getShardSpec()); + // The shardSpecs for partitinoing and publishing can be different if isExtendableShardSpecs = true. + sequenceNameToSegmentId.put(getSequenceName(interval, segmentIdentifier.getShardSpec()), segmentIdentifier); + } + } + shardSpecs = new ShardSpecs(shardSpecMap); } - @Override - Map> getIntervalToSegmentIds() throws IOException + private Map> getIntervalToSegmentIds() throws IOException { final Map intervalToVersion = getToolbox().getTaskActionClient() .submit(new LockListAction()) @@ -95,4 +127,53 @@ private static String findVersion(Map intervalToVersion, Inter .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> 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/CachingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java deleted file mode 100644 index 10d29fde5d61..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.common.task; - -import org.apache.druid.data.input.InputRow; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs; -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; - -/** - * Allocates all necessary segments at the beginning and reuse them. - */ -abstract class CachingSegmentAllocator implements IndexTaskSegmentAllocator -{ - private final TaskToolbox toolbox; - private final String taskId; - private final String dataSource; - private final Map> allocateSpec; - @Nullable - private final ShardSpecs shardSpecs; - - // sequenceName -> segmentId - private final Map sequenceNameToSegmentId; - - CachingSegmentAllocator( - TaskToolbox toolbox, - String taskId, - String dataSource, - Map> allocateSpec - ) throws IOException - { - this.toolbox = toolbox; - this.taskId = taskId; - this.dataSource = dataSource; - this.allocateSpec = allocateSpec; - this.sequenceNameToSegmentId = new HashMap<>(); - - final Map> intervalToIds = getIntervalToSegmentIds(); - final Map> shardSpecMap = new HashMap<>(); - - for (Map.Entry> entry : intervalToIds.entrySet()) { - final Interval interval = entry.getKey(); - final List idsPerInterval = intervalToIds.get(interval); - - for (SegmentIdWithShardSpec segmentIdentifier : idsPerInterval) { - shardSpecMap.computeIfAbsent(interval, k -> new ArrayList<>()).add(segmentIdentifier.getShardSpec()); - // The shardSpecs for partitinoing and publishing can be different if isExtendableShardSpecs = true. - sequenceNameToSegmentId.put(getSequenceName(interval, segmentIdentifier.getShardSpec()), segmentIdentifier); - } - } - shardSpecs = new ShardSpecs(shardSpecMap); - } - - abstract Map> getIntervalToSegmentIds() throws IOException; - - TaskToolbox getToolbox() - { - return toolbox; - } - - String getTaskId() - { - return taskId; - } - - String getDataSource() - { - return dataSource; - } - - Map> 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) - { - 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 88c0c43e72d2..5d3bdf842b55 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 @@ -222,6 +222,7 @@ public AggregatorFactory[] getMetricsSpec() @JsonProperty @Nullable + @Override public Granularity getSegmentGranularity() { return segmentGranularity; @@ -257,44 +258,28 @@ public int getPriority() public boolean isReady(TaskActionClient taskActionClient) throws Exception { final List segments = segmentProvider.checkAndGetSegments(taskActionClient); - return tryLockWithSegments(taskActionClient, segments, true); + return determineLockGranularityandTryLockWithSegments(taskActionClient, segments); } @Override - public boolean requireLockInputSegments() + public boolean requireLockExistingSegments() { return true; } @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); } - @Override - public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) - { - return segmentGranularity != null; - } - @Override public boolean isPerfectRollup() { return tuningConfig != null && tuningConfig.isForceGuaranteedRollup(); } - @Override - public Granularity getSegmentGranularity(Interval interval) - { - if (segmentGranularity == null) { - return GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); - } else { - return segmentGranularity; - } - } - @Override public TaskStatus run(final TaskToolbox toolbox) throws Exception { @@ -315,16 +300,16 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .range(0, ingestionSpecs.size()) .mapToObj(i -> new IndexTask( createIndexTaskSpecId(i), - getGroupId(), - getTaskResource(), - getDataSource(), + getGroupId(), + getTaskResource(), + getDataSource(), ingestionSpecs.get(i), - getContext(), - authorizerMapper, - chatHandlerProvider, - rowIngestionMetersFactory - )) - .collect(Collectors.toList()); + getContext(), + authorizerMapper, + chatHandlerProvider, + rowIngestionMetersFactory + )) + .collect(Collectors.toList()); } if (indexTaskSpecs.isEmpty()) { @@ -340,10 +325,15 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception 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) { 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 0efa7359ccac..9b2fb8abb49a 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 @@ -56,8 +56,6 @@ 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; @@ -209,24 +207,17 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception } @Override - public boolean requireLockInputSegments() + public boolean requireLockExistingSegments() { throw new UnsupportedOperationException(); } @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) { throw new UnsupportedOperationException(); } - @Override - public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) - { - final Granularity segmentGranularity = spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); - return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); - } - @Override public boolean isPerfectRollup() { @@ -235,14 +226,9 @@ public boolean isPerfectRollup() @Nullable @Override - public Granularity getSegmentGranularity(Interval interval) + public Granularity getSegmentGranularity() { - final GranularitySpec granularitySpec = spec.getDataSchema().getGranularitySpec(); - if (granularitySpec instanceof ArbitraryGranularitySpec) { - return null; - } else { - return granularitySpec.getSegmentGranularity(); - } + return spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); } @JsonProperty("spec") 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 94fe68b67d8a..8cf019a29036 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 @@ -42,17 +42,15 @@ 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.TaskRealtimeMetricsMonitorBuilder; import org.apache.druid.indexing.common.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; -import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; -import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; @@ -109,7 +107,6 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -121,7 +118,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; public class IndexTask extends AbstractBatchIndexTask implements ChatHandler { @@ -158,22 +154,22 @@ private static String makeGroupId(boolean isAppendToExisting, String dataSource) private final Optional chatHandlerProvider; @JsonIgnore - private FireDepartmentMetrics buildSegmentsFireDepartmentMetrics; + private final RowIngestionMeters determinePartitionsMeters; @JsonIgnore - private CircularBuffer buildSegmentsSavedParseExceptions; + private final RowIngestionMeters buildSegmentsMeters; @JsonIgnore - private CircularBuffer determinePartitionsSavedParseExceptions; + private FireDepartmentMetrics buildSegmentsFireDepartmentMetrics; @JsonIgnore - private String errorMsg; + private CircularBuffer buildSegmentsSavedParseExceptions; @JsonIgnore - private final RowIngestionMeters determinePartitionsMeters; + private CircularBuffer determinePartitionsSavedParseExceptions; @JsonIgnore - private final RowIngestionMeters buildSegmentsMeters; + private String errorMsg; @JsonCreator public IndexTask( @@ -250,26 +246,18 @@ public String getType() @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - final Optional> intervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals(); - - if (intervals.isPresent()) { - return tryLockIfNecessary(taskActionClient, intervals.get()); - } else { - return true; - } + return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.dataSchema.getGranularitySpec()); } @Override - public boolean requireLockInputSegments() + public boolean requireLockExistingSegments() { return isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig) || !ingestionSchema.ioConfig.isAppendToExisting(); } @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { return findInputSegments( @@ -280,61 +268,15 @@ public List findInputSegments(TaskActionClient taskActionClient, Li ); } - public static List findInputSegments( - String dataSource, - TaskActionClient actionClient, - List intervalsToFind, - FirehoseFactory firehoseFactory - ) throws IOException - { - if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { - final List 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 inputSegmentIds = inputSegments.stream() - .map(WindowedSegmentId::getSegmentId) - .collect(Collectors.toList()); - final List 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, intervalsToFind)); - } - } - - @Override - public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) - { - final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); - return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); - } - @Override public boolean isPerfectRollup() { - return ingestionSchema.tuningConfig.isForceGuaranteedRollup(); + return isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig); } @Nullable @Override - public Granularity getSegmentGranularity(Interval interval) + public Granularity getSegmentGranularity() { final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); if (granularitySpec instanceof ArbitraryGranularitySpec) { @@ -344,18 +286,6 @@ public Granularity getSegmentGranularity(Interval interval) } } - private boolean tryLockIfNecessary(TaskActionClient actionClient, Collection 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 tryLockWithIntervals(actionClient, new ArrayList<>(intervals), true); - } - @GET @Path("/unparseableEvents") @Produces(MediaType.APPLICATION_JSON) @@ -519,14 +449,9 @@ public TaskStatus run(final TaskToolbox toolbox) ); final List allocateIntervals = new ArrayList<>(allocateSpec.keySet()); - // get locks for found shardSpec intervals - if (!tryLockIfNecessary(toolbox.getTaskActionClient(), allocateIntervals)) { - throw new ISE("Failed to get a lock for segments"); - } - final DataSchema dataSchema; if (determineIntervals) { - if (!tryLockWithIntervals(toolbox.getTaskActionClient(), allocateIntervals, true)) { + if (!determineLockGranularityandTryLock(toolbox.getTaskActionClient(), allocateIntervals)) { throw new ISE("Failed to get locks for intervals[%s]", allocateIntervals); } @@ -870,14 +795,14 @@ private IndexTaskSegmentAllocator createSegmentAllocator( Map> allocateSpec ) throws IOException { - if (!isGuaranteedRollup(ingestionSchema.ioConfig, ingestionSchema.tuningConfig) - && (ingestionSchema.ioConfig.isAppendToExisting() || !isChangeSegmentGranularity())) { + if (ingestionSchema.ioConfig.isAppendToExisting() || isUseSegmentLock()) { return new RemoteSegmentAllocator( toolbox, getId(), dataSchema, - hasInputSegments() && !isChangeSegmentGranularity(), - getAllOverwritingSegmentMeta() + getNonNullSegmentLockHelper(), + isUseSegmentLock() ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK, + ingestionSchema.ioConfig.isAppendToExisting() ); } else { // We use the timeChunk lock and don't have to ask the overlord to create segmentIds. @@ -957,7 +882,7 @@ private TaskStatus generateAndPublishSegments( final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { - driver.startJob(null); + driver.startJob(); while (firehose.hasMore()) { try { @@ -982,17 +907,8 @@ private TaskStatus generateAndPublishSegments( continue; } - final String sequenceName; - - if (isGuaranteedRollup) { - // Sequence name is based solely on the shardSpec, and there will only be one segment per sequence. - final Interval interval = optInterval.get(); - sequenceName = segmentAllocator.getSequenceName(interval, inputRow); - } else { - // Segments are created as needed, using a single sequence name. They may be allocated from the overlord - // (in append mode) or may be created on our own authority (in overwrite mode). - sequenceName = getId(); - } + final Interval interval = optInterval.get(); + final String sequenceName = segmentAllocator.getSequenceName(interval, inputRow); final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); if (addResult.isOk()) { @@ -1023,8 +939,8 @@ private TaskStatus generateAndPublishSegments( log.info("Pushed segments[%s]", pushed.getSegments()); // Probably we can publish atomicUpdateGroup along with segments. - final Set inputSegments = !isGuaranteedRollup && hasInputSegments() && !isChangeSegmentGranularity() - ? getAllInputSegments() + final Set inputSegments = isUseSegmentLock() + ? getNonNullSegmentLockHelper().getLockedExistingSegments() : null; final SegmentsAndMetadata published = awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java index 2ebb9f1dad72..d997675f5962 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java @@ -28,5 +28,15 @@ */ interface IndexTaskSegmentAllocator extends SegmentAllocator { + /** + * SequenceName is the key to create the segmentId. If previousSegmentId is given, {@link SegmentAllocator} allocates + * segmentId depending on sequenceName and previousSegmentId. If it's missing, it allocates segmentId using + * sequenceName and interval. For {@link IndexTask}, it always provides the previousSegmentId to + * SegmentAllocator. + * See {@link org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator#allocatePendingSegment} for details. + * + * Implementations should return the correct sequenceName based on the given interval and inputRow, which is passed + * to SegmentAllocator. + */ String getSequenceName(Interval interval, InputRow inputRow); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java index 44e63b05d034..3c8e99ab536d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java @@ -86,6 +86,8 @@ class LocalSegmentAllocator implements IndexTaskSegmentAllocator @Override public String getSequenceName(Interval interval, InputRow inputRow) { + // Segments are created as needed, using a single sequence name. They may be allocated from the overlord + // (in append mode) or may be created on our own authority (in overwrite mode). return taskId; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java index d3ac959d31e0..4fc117df88fb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RemoteSegmentAllocator.java @@ -21,9 +21,10 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; +import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; -import org.apache.druid.indexing.common.task.AbstractBatchIndexTask.OverwritingRootGenerationPartitions; +import org.apache.druid.indexing.common.task.SegmentLockHelper.OverwritingRootGenerationPartitions; import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -34,12 +35,11 @@ import org.joda.time.Interval; import java.io.IOException; -import java.util.Map; /** * Segment allocator which allocates new segments using the overlord per request. */ -class RemoteSegmentAllocator implements IndexTaskSegmentAllocator +public class RemoteSegmentAllocator implements IndexTaskSegmentAllocator { private final String taskId; private final ActionBasedSegmentAllocator internalAllocator; @@ -48,8 +48,9 @@ class RemoteSegmentAllocator implements IndexTaskSegmentAllocator final TaskToolbox toolbox, final String taskId, final DataSchema dataSchema, - final boolean needMinorOverwrite, - final Map overwritingSegmentMetaMap + final SegmentLockHelper segmentLockHelper, + final LockGranularity lockGranularity, + final boolean appendToExisting ) { this.taskId = taskId; @@ -61,30 +62,46 @@ class RemoteSegmentAllocator implements IndexTaskSegmentAllocator final Interval interval = granularitySpec .bucketInterval(row.getTimestamp()) .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); - final ShardSpecFactory shardSpecFactory; - if (needMinorOverwrite) { - final OverwritingRootGenerationPartitions overwritingSegmentMeta = overwritingSegmentMetaMap.get(interval); - if (overwritingSegmentMeta == null) { - throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); - } - shardSpecFactory = new NumberedOverwritingShardSpecFactory( - overwritingSegmentMeta.getStartRootPartitionId(), - overwritingSegmentMeta.getEndRootPartitionId(), - overwritingSegmentMeta.getMinorVersionForNewSegments() + if (lockGranularity == LockGranularity.TIME_CHUNK) { + return new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck, + NumberedShardSpecFactory.instance(), + lockGranularity ); } else { - shardSpecFactory = NumberedShardSpecFactory.instance(); + final ShardSpecFactory shardSpecFactory; + if (segmentLockHelper.hasLockedExistingSegments() && !appendToExisting) { + final OverwritingRootGenerationPartitions overwritingSegmentMeta = segmentLockHelper + .getOverwritingRootGenerationPartition(interval); + if (overwritingSegmentMeta == null) { + throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); + } + shardSpecFactory = new NumberedOverwritingShardSpecFactory( + overwritingSegmentMeta.getStartRootPartitionId(), + overwritingSegmentMeta.getEndRootPartitionId(), + overwritingSegmentMeta.getMinorVersionForNewSegments() + ); + } else { + shardSpecFactory = NumberedShardSpecFactory.instance(); + } + return new SegmentAllocateAction( + schema.getDataSource(), + row.getTimestamp(), + schema.getGranularitySpec().getQueryGranularity(), + schema.getGranularitySpec().getSegmentGranularity(), + sequenceName, + previousSegmentId, + skipSegmentLineageCheck, + shardSpecFactory, + lockGranularity + ); } - return new SegmentAllocateAction( - schema.getDataSource(), - row.getTimestamp(), - schema.getGranularitySpec().getQueryGranularity(), - schema.getGranularitySpec().getSegmentGranularity(), - sequenceName, - previousSegmentId, - skipSegmentLineageCheck, - shardSpecFactory - ); } ); } @@ -103,6 +120,8 @@ public SegmentIdWithShardSpec allocate( @Override public String getSequenceName(Interval interval, InputRow inputRow) { + // Segments are created as needed, using a single sequence name. They may be allocated from the overlord + // (in append mode) or may be created on our own authority (in overwrite mode). return taskId; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java new file mode 100644 index 000000000000..c59e5af66a9d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java @@ -0,0 +1,349 @@ +/* + * 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.indexing.common.SegmentLock; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.actions.SegmentListUsedAction; +import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction; +import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; +import org.apache.druid.indexing.firehose.WindowedSegmentId; +import org.apache.druid.indexing.overlord.LockResult; +import org.apache.druid.java.util.common.ISE; +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.io.Closer; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class SegmentLockHelper +{ + private final String dataSource; + private final Map overwritingRootGenPartitions = new HashMap<>(); + private final Set lockedExistingSegments = new HashSet<>(); + @Nullable + private Granularity knownSegmentGranularity; + + public static class OverwritingRootGenerationPartitions + { + private final int startRootPartitionId; + private final int endRootPartitionId; + private final short maxMinorVersion; + + private OverwritingRootGenerationPartitions(int startRootPartitionId, int endRootPartitionId, short maxMinorVersion) + { + this.startRootPartitionId = startRootPartitionId; + this.endRootPartitionId = endRootPartitionId; + this.maxMinorVersion = maxMinorVersion; + } + + public int getStartRootPartitionId() + { + return startRootPartitionId; + } + + public int getEndRootPartitionId() + { + return endRootPartitionId; + } + + public short getMinorVersionForNewSegments() + { + return (short) (maxMinorVersion + 1); + } + } + + public SegmentLockHelper(String dataSource) + { + this.dataSource = dataSource; + } + + public boolean hasLockedExistingSegments() + { + return !lockedExistingSegments.isEmpty(); + } + + public boolean hasOverwritingRootGenerationPartition(Interval interval) + { + return overwritingRootGenPartitions.containsKey(interval); + } + + public Set getLockedExistingSegments() + { + return Collections.unmodifiableSet(lockedExistingSegments); + } + + public OverwritingRootGenerationPartitions getOverwritingRootGenerationPartition(Interval interval) + { + return overwritingRootGenPartitions.get(interval); + } + + @Nullable + public Granularity getKnownSegmentGranularity() + { + return knownSegmentGranularity; + } + + public boolean tryLockExistingSegments( + TaskActionClient actionClient, + List intervalsToFind // intervals must be aligned with the segment granularity of existing segments + ) throws IOException + { + final List segments = actionClient.submit( + new SegmentListUsedAction( + dataSource, + null, + intervalsToFind + ) + ); + return verifyAndLockExistingSegments(actionClient, segments); + } + + public boolean tryLockExistingSegments( + TaskActionClient actionClient, + IngestSegmentFirehoseFactory firehoseFactory + ) throws IOException + { + return verifyAndLockExistingSegments(actionClient, findExistingSegments(actionClient, firehoseFactory)); + } + + public boolean verifyAndLockExistingSegments(TaskActionClient actionClient, List segments) + throws IOException + { + final List segmentsToLock = segments.stream() + .filter(segment -> !lockedExistingSegments.contains(segment)) + .collect(Collectors.toList()); + if (segmentsToLock.isEmpty()) { + return true; + } + + verifySegmentGranularity(segmentsToLock); + return tryLockSegmentsIfNeeded(actionClient, segmentsToLock); + } + + private void verifySegmentGranularity(List segments) + { + final Granularity granularityFromSegments = AbstractBatchIndexTask.findGranularityFromSegments(segments); + if (granularityFromSegments != null) { + if (knownSegmentGranularity == null) { + knownSegmentGranularity = granularityFromSegments; + } else { + if (!knownSegmentGranularity.equals(granularityFromSegments)) { + throw new ISE( + "Found a different granularity from knownSegmentGranularity[%s] in segments[%s]", + knownSegmentGranularity, + segments + ); + } + final List nonAlignedSegments = segments + .stream() + .filter(segment -> !knownSegmentGranularity.isAligned(segment.getInterval())) + .collect(Collectors.toList()); + + if (!nonAlignedSegments.isEmpty()) { + throw new ISE( + "Non-aligned segments[%s] for granularity[%s]", + nonAlignedSegments.stream().map(DataSegment::getId).collect(Collectors.toList()), + knownSegmentGranularity + ); + } + } + } else { + throw new ISE( + "Found different granularities in segments[%s]", + segments.stream().map(DataSegment::getId).collect(Collectors.toList()) + ); + } + } + + private boolean tryLockSegmentsIfNeeded(TaskActionClient actionClient, List segments) throws IOException + { + final Map> intervalToSegments = groupSegmentsByInterval(segments); + final Closer lockCloserOnError = Closer.create(); + for (List segmentsInInterval : intervalToSegments.values()) { + for (DataSegment segment : segmentsInInterval) { + final List lockResults = actionClient.submit( + new SegmentLockTryAcquireAction( + TaskLockType.EXCLUSIVE, + segment.getInterval(), + segment.getMajorVersion(), Collections.singleton(segment.getShardSpec().getPartitionNum()) + ) + ); + lockResults.stream() + .filter(LockResult::isOk) + .map(result -> (SegmentLock) result.getTaskLock()) + .forEach(segmentLock -> lockCloserOnError.register(() -> actionClient.submit( + new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) + ))); + if (lockResults.stream().anyMatch(result -> !result.isOk())) { + lockCloserOnError.close(); + return false; + } + lockedExistingSegments.add(segment); + } + verifyAndFindRootPartitionRangeAndMinorVersion(segmentsInInterval); + } + return true; + } + + /** + * This method is called when the task overwrites existing segments with segment locks. It verifies the input segments + * can be locked together, so that output segments can overshadow existing ones properly. + *

+ * This method checks two things: + *

+ * - Are rootPartition range of inputSegments adjacent? Two rootPartition ranges are adjacent if they are consecutive. + * - All atomicUpdateGroups of inputSegments must be full. (See {@code AtomicUpdateGroup#isFull()}). + */ + private void verifyAndFindRootPartitionRangeAndMinorVersion(List inputSegments) + { + if (inputSegments.isEmpty()) { + return; + } + + final List sortedSegments = new ArrayList<>(inputSegments); + sortedSegments.sort((s1, s2) -> { + if (s1.getStartRootPartitionId() != s2.getStartRootPartitionId()) { + return Integer.compare(s1.getStartRootPartitionId(), s2.getStartRootPartitionId()); + } else { + return Integer.compare(s1.getEndRootPartitionId(), s2.getEndRootPartitionId()); + } + }); + verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(sortedSegments); + final Interval interval = sortedSegments.get(0).getInterval(); + final short prevMaxMinorVersion = (short) sortedSegments + .stream() + .mapToInt(DataSegment::getMinorVersion) + .max() + .orElseThrow(() -> new ISE("Empty inputSegments")); + + overwritingRootGenPartitions.put( + interval, + new OverwritingRootGenerationPartitions( + sortedSegments.get(0).getStartRootPartitionId(), + sortedSegments.get(sortedSegments.size() - 1).getEndRootPartitionId(), + prevMaxMinorVersion + ) + ); + } + + public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List sortedSegments) + { + if (sortedSegments.isEmpty()) { + return; + } + + Preconditions.checkArgument( + sortedSegments.stream().allMatch(segment -> segment.getInterval().equals(sortedSegments.get(0).getInterval())) + ); + + short atomicUpdateGroupSize = 1; + // sanity check + for (int i = 0; i < sortedSegments.size() - 1; i++) { + final DataSegment curSegment = sortedSegments.get(i); + final DataSegment nextSegment = sortedSegments.get(i + 1); + if (curSegment.getStartRootPartitionId() == nextSegment.getStartRootPartitionId() + && curSegment.getEndRootPartitionId() == nextSegment.getEndRootPartitionId()) { + // Input segments should have the same or consecutive rootPartition range + if (curSegment.getMinorVersion() != nextSegment.getMinorVersion() + || curSegment.getAtomicUpdateGroupSize() != nextSegment.getAtomicUpdateGroupSize()) { + throw new ISE( + "segment[%s] and segment[%s] have the same rootPartitionRange, but different minorVersion or atomicUpdateGroupSize", + curSegment, + nextSegment + ); + } + atomicUpdateGroupSize++; + } else { + if (curSegment.getEndRootPartitionId() != nextSegment.getStartRootPartitionId()) { + throw new ISE("Can't compact segments of non-consecutive rootPartition range"); + } + if (atomicUpdateGroupSize != curSegment.getAtomicUpdateGroupSize()) { + throw new ISE("All atomicUpdateGroup must be compacted together"); + } + atomicUpdateGroupSize = 1; + } + } + if (atomicUpdateGroupSize != sortedSegments.get(sortedSegments.size() - 1).getAtomicUpdateGroupSize()) { + throw new ISE("All atomicUpdateGroup must be compacted together"); + } + } + + private List findExistingSegments( + TaskActionClient actionClient, + IngestSegmentFirehoseFactory firehoseFactory + ) throws IOException + { + final List inputSegments = firehoseFactory.getSegments(); + if (inputSegments == null) { + final Interval inputInterval = Preconditions.checkNotNull(firehoseFactory.getInterval(), "input interval"); + + return actionClient.submit( + new SegmentListUsedAction(dataSource, null, Collections.singletonList(inputInterval)) + ); + } else { + final List inputSegmentIds = inputSegments.stream() + .map(WindowedSegmentId::getSegmentId) + .collect(Collectors.toList()); + final Set dataSegmentsInIntervals = new HashSet<>( + 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()); + } + } + + private static Map> groupSegmentsByInterval(List segments) + { + final Map> map = new HashMap<>(); + for (DataSegment segment : segments) { + map.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment); + } + return map; + } + + private static Granularity granularityOfInterval(Interval interval) + { + return GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index c70b37df6ab5..3aa617d16faa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -36,9 +36,11 @@ public class Tasks public static final int DEFAULT_MERGE_TASK_PRIORITY = 25; public static final int DEFAULT_TASK_PRIORITY = 0; public static final long DEFAULT_LOCK_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); + public static final boolean DEFAULT_FORCE_TIME_CHUNK_LOCK = true; public static final String PRIORITY_KEY = "priority"; public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout"; + public static final String FORCE_TIME_CHUNK_LOCK_KEY = "forceTimeChunkLock"; public static SortedSet computeCompactIntervals(SortedSet intervals) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 0252663b820e..efa7a329bfc0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -32,6 +32,7 @@ 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.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SurrogateAction; @@ -40,6 +41,8 @@ import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.SegmentLockHelper; +import org.apache.druid.indexing.common.task.SegmentLockHelper.OverwritingRootGenerationPartitions; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.ISE; @@ -47,12 +50,12 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; 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.java.util.common.parsers.ParseException; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeIOConfig; -import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireDepartment; @@ -77,13 +80,11 @@ import javax.annotation.Nullable; import java.io.File; 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.SortedSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @@ -151,24 +152,12 @@ public String getType() } @Override - public boolean isReady(TaskActionClient taskActionClient) + public boolean isReady(TaskActionClient taskActionClient) throws IOException { - final Optional> intervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals(); - - return !intervals.isPresent() || checkLockAcquired(taskActionClient, intervals.get()); - } - - private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet intervals) - { - try { - return tryLockWithIntervals(actionClient, new ArrayList<>(intervals), false); - } - catch (Exception e) { - log.error(e, "Failed to acquire locks for intervals[%s]", intervals); - return false; + if (!ingestionSchema.getDataSchema().getGranularitySpec().bucketIntervals().isPresent()) { + addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } + return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.getDataSchema().getGranularitySpec()); } @JsonProperty @@ -213,7 +202,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ); // Find inputSegments overshadowed by pushedSegments - final Set allSegments = new HashSet<>(getAllInputSegments()); + final Set allSegments = new HashSet<>(getNonNullSegmentLockHelper().getLockedExistingSegments()); allSegments.addAll(pushedSegments); final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(allSegments); final Set oldSegments = timeline.findFullyOvershadowed() @@ -227,16 +216,16 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception } @Override - public boolean requireLockInputSegments() + public boolean requireLockExistingSegments() { return !ingestionSchema.getIOConfig().isAppendToExisting(); } @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - return IndexTask.findInputSegments( + return findInputSegments( getDataSource(), taskActionClient, intervals, @@ -244,13 +233,6 @@ public List findInputSegments(TaskActionClient taskActionClient, Li ); } - @Override - public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) - { - final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); - return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); - } - @Override public boolean isPerfectRollup() { @@ -259,14 +241,9 @@ public boolean isPerfectRollup() @Nullable @Override - public Granularity getSegmentGranularity(Interval interval) + public Granularity getSegmentGranularity() { - final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - if (granularitySpec instanceof ArbitraryGranularitySpec) { - return null; - } else { - return granularitySpec.getSegmentGranularity(); - } + return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); } @VisibleForTesting @@ -280,11 +257,6 @@ private class WrappingSegmentAllocator implements SegmentAllocator private final TaskToolbox toolbox; private final ParallelIndexTaskClient taskClient; - /** - * This internalAllocator is initialized lazily to make sure that {@link #isChangeSegmentGranularity()} is called - * after the lock is properly acquired. Note that locks can be acquired after the task is started if the interval is - * not specified in {@link GranularitySpec}. - */ private SegmentAllocator internalAllocator; private WrappingSegmentAllocator(TaskToolbox toolbox, ParallelIndexTaskClient taskClient) @@ -309,18 +281,37 @@ public SegmentIdWithShardSpec allocate( private SegmentAllocator createSegmentAllocator() { - if (ingestionSchema.getIOConfig().isAppendToExisting() || !isChangeSegmentGranularity()) { + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + final SegmentLockHelper segmentLockHelper = getNonNullSegmentLockHelper(); + if (ingestionSchema.getIOConfig().isAppendToExisting() || isUseSegmentLock()) { return new ActionBasedSegmentAllocator( toolbox.getTaskActionClient(), ingestionSchema.getDataSchema(), (schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> { - final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); final Interval interval = granularitySpec .bucketInterval(row.getTimestamp()) .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); + final Granularity granularityFromInterval = GranularityType.fromPeriod(interval.toPeriod()) + .getDefaultGranularity(); final ShardSpecFactory shardSpecFactory; - if (hasInputSegments() && !isChangeSegmentGranularity()) { - final OverwritingRootGenerationPartitions overwritingSegmentMeta = getOverwritingSegmentMeta(interval); + if (segmentLockHelper.hasOverwritingRootGenerationPartition(interval)) { + if (segmentLockHelper.getKnownSegmentGranularity() != null + && !segmentLockHelper.getKnownSegmentGranularity().equals(granularityFromInterval)) { + throw new ISE( + "Different segment granularity is detected. knownGranularity[%s], granularityFromInterval[%s]", + segmentLockHelper.getKnownSegmentGranularity(), + granularityFromInterval + ); + } + final boolean lockedExistingSegments = segmentLockHelper.tryLockExistingSegments( + toolbox.getTaskActionClient(), + Collections.singletonList(interval) + ); + if (!lockedExistingSegments) { + throw new ISE("Failed to lock segments in interval[%s]", interval); + } + final OverwritingRootGenerationPartitions overwritingSegmentMeta = segmentLockHelper + .getOverwritingRootGenerationPartition(interval); if (overwritingSegmentMeta == null) { throw new ISE("Can't find overwritingSegmentMeta for interval[%s]", interval); } @@ -343,7 +334,8 @@ private SegmentAllocator createSegmentAllocator() sequenceName, previousSegmentId, skipSegmentLineageCheck, - shardSpecFactory + shardSpecFactory, + isUseSegmentLock() ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK ) ); } @@ -410,7 +402,7 @@ private Set generateAndPushSegments( final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { - driver.startJob(null); + driver.startJob(); final Set pushedSegments = new HashSet<>(); @@ -437,12 +429,6 @@ private Set generateAndPushSegments( fireDepartmentMetrics.incrementThrownAway(); continue; } - } else { - final Granularity segmentGranularity = findSegmentGranularity(granularitySpec); - final Interval timeChunk = segmentGranularity.bucket(inputRow.getTimestamp()); - if (!tryLockWithIntervals(toolbox.getTaskActionClient(), Collections.singletonList(timeChunk), false)) { - throw new ISE("Failed to get locks for interval[%s]", timeChunk); - } } // Segments are created as needed, using a single sequence name. They may be allocated from the overlord diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index e040eff95645..3067841136ab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -33,9 +33,11 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.Counters; import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; @@ -51,7 +53,6 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; 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.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -77,7 +78,6 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -217,18 +217,18 @@ void setRunner(ParallelIndexTaskRunner runner) @Override public boolean isReady(TaskActionClient taskActionClient) throws Exception { - final Optional> intervals = ingestionSchema.getDataSchema() - .getGranularitySpec() - .bucketIntervals(); - - return !intervals.isPresent() || tryLockWithIntervals(taskActionClient, new ArrayList<>(intervals.get()), true); + if (!ingestionSchema.getDataSchema().getGranularitySpec().bucketIntervals().isPresent() + && !ingestionSchema.getIOConfig().isAppendToExisting()) { + addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); + } + return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.getDataSchema().getGranularitySpec()); } @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - return IndexTask.findInputSegments( + return findInputSegments( getDataSource(), taskActionClient, intervals, @@ -237,18 +237,11 @@ public List findInputSegments(TaskActionClient taskActionClient, Li } @Override - public boolean requireLockInputSegments() + public boolean requireLockExistingSegments() { return !ingestionSchema.getIOConfig().isAppendToExisting(); } - @Override - public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) - { - final Granularity segmentGranularity = ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); - return intervalOfExistingSegments.stream().anyMatch(interval -> !segmentGranularity.match(interval)); - } - @Override public boolean isPerfectRollup() { @@ -257,14 +250,9 @@ public boolean isPerfectRollup() @Nullable @Override - public Granularity getSegmentGranularity(Interval interval) + public Granularity getSegmentGranularity() { - final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - if (granularitySpec instanceof ArbitraryGranularitySpec) { - return null; - } else { - return granularitySpec.getSegmentGranularity(); - } + return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); } @Override @@ -331,9 +319,9 @@ private TaskStatus runParallel(TaskToolbox toolbox) throws Exception return TaskStatus.fromCode(getId(), Preconditions.checkNotNull(runner, "runner").run()); } - private TaskStatus runSequential(TaskToolbox toolbox) + private TaskStatus runSequential(TaskToolbox toolbox) throws Exception { - return new IndexTask( + final IndexTask indexTask = new IndexTask( getId(), getGroupId(), getTaskResource(), @@ -347,7 +335,12 @@ private TaskStatus runSequential(TaskToolbox toolbox) authorizerMapper, chatHandlerProvider, rowIngestionMetersFactory - ).run(toolbox); + ); + if (indexTask.isReady(toolbox.getTaskActionClient())) { + return indexTask.run(toolbox); + } else { + return TaskStatus.failure(getId()); + } } private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningConfig tuningConfig) @@ -406,38 +399,86 @@ public Response allocateSegment(DateTime timestamp, @Context final HttpServletRe } } + // TODO: add test for timeChunkLock without bucket intervals in granularitySpec + + /** + * Allocate a new segment for the given timestamp locally. + * Since the segments returned by this method overwrites any existing segments, this method should be called only + * when the {@link org.apache.druid.indexing.common.LockGranularity} is {@code TIME_CHUNK}. + */ @VisibleForTesting SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException { final String dataSource = getDataSource(); final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec(); + final Optional> bucketIntervals = granularitySpec.bucketIntervals(); + // List locks whenever allocating a new segment because locks might be revoked and no longer valid. - final Map versions = toolbox + final List locks = toolbox .getTaskActionClient() - .submit(new LockListAction()) + .submit(new LockListAction()); + final TaskLock revokedLock = locks.stream().filter(TaskLock::isRevoked).findAny().orElse(null); + if (revokedLock != null) { + throw new ISE("Lock revoked: [%s]", revokedLock); + } + final Map versions = locks .stream() .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - final Interval interval = granularitySpec - .bucketInterval(timestamp) - .or(granularitySpec.getSegmentGranularity().bucket(timestamp)); + Interval interval; + String version; + if (bucketIntervals.isPresent()) { + // If granularity spec has explicit intervals, we just need to find the version accociated to the interval. + // This is because we should have gotten all required locks up front when the task starts up. + final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); + if (!maybeInterval.isPresent()) { + throw new IAE("Could not find interval for timestamp [%s]", timestamp); + } + + interval = maybeInterval.get(); + if (!bucketIntervals.get().contains(interval)) { + throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec); + } + + version = findVersion(versions, interval); + if (version == null) { + throw new ISE("Cannot find a version for interval[%s]", interval); + } + } else { + // We don't have explicit intervals. We can use the segment granularity to figure out what + // interval we need, but we might not have already locked it. + interval = granularitySpec.getSegmentGranularity().bucket(timestamp); + version = findVersion(versions, interval); + if (version == null) { + // We don't have a lock for this interval, so we should lock it now. + final TaskLock lock = Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval) + ), + "Cannot acquire a lock for interval[%s]", + interval + ); + version = lock.getVersion(); + } + } final int partitionNum = Counters.getAndIncrementInt(partitionNumCountersPerInterval, interval); return new SegmentIdWithShardSpec( dataSource, interval, - findVersion(versions, interval), + version, new NumberedShardSpec(partitionNum, 0) ); } + @Nullable private static String findVersion(Map versions, Interval interval) { return versions.entrySet().stream() .filter(entry -> entry.getKey().contains(interval)) .map(Entry::getValue) .findFirst() - .orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval)); + .orElse(null); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java index 1edb3f7dc57e..7ce31de29086 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockRequestForNewSegment.java @@ -32,6 +32,7 @@ public class LockRequestForNewSegment implements LockRequest { + private final LockGranularity lockGranularity; private final TaskLockType lockType; private final String groupId; private final String dataSource; @@ -43,7 +44,10 @@ public class LockRequestForNewSegment implements LockRequest private final String previsousSegmentId; private final boolean skipSegmentLineageCheck; + private String version; + public LockRequestForNewSegment( + LockGranularity lockGranularity, TaskLockType lockType, String groupId, String dataSource, @@ -55,6 +59,7 @@ public LockRequestForNewSegment( boolean skipSegmentLineageCheck ) { + this.lockGranularity = lockGranularity; this.lockType = lockType; this.groupId = groupId; this.dataSource = dataSource; @@ -68,6 +73,7 @@ public LockRequestForNewSegment( @VisibleForTesting public LockRequestForNewSegment( + LockGranularity lockGranularity, TaskLockType lockType, Task task, Interval interval, @@ -78,6 +84,7 @@ public LockRequestForNewSegment( ) { this( + lockGranularity, lockType, task.getGroupId(), task.getDataSource(), @@ -93,7 +100,7 @@ public LockRequestForNewSegment( @Override public LockGranularity getGranularity() { - return LockGranularity.SEGMENT; + return lockGranularity; } @Override @@ -134,7 +141,10 @@ public ShardSpecFactory getShardSpecFactory() @Override public String getVersion() { - return DateTimes.nowUtc().toString(); + if (version == null) { + version = DateTimes.nowUtc().toString(); + } + return version; } @Override @@ -146,7 +156,10 @@ public boolean isRevoked() @Override public TaskLock toLock() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException( + "This lockRequest must be converted to SpecificSegmentLockRequest or TimeChunkLockRequest first" + + " to convert to TaskLock" + ); } public String getSequenceName() @@ -169,7 +182,8 @@ public boolean isSkipSegmentLineageCheck() public String toString() { return "LockRequestForNewSegment{" + - "lockType=" + lockType + + "lockGranularity=" + lockGranularity + + ", lockType=" + lockType + ", groupId='" + groupId + '\'' + ", dataSource='" + dataSource + '\'' + ", interval=" + interval + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index caba20941e79..3ca7173938ad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -344,22 +344,39 @@ public LockResult tryLock(final Task task, final LockRequest request) } Preconditions.checkArgument(request.getInterval().toDurationMillis() > 0, "interval empty"); - final SegmentIdWithShardSpec newSegmentId; + SegmentIdWithShardSpec newSegmentId = null; final LockRequest convertedRequest; if (request instanceof LockRequestForNewSegment) { final LockRequestForNewSegment lockRequestForNewSegment = (LockRequestForNewSegment) request; - newSegmentId = allocateSegmentId(lockRequestForNewSegment); - if (newSegmentId == null) { - return LockResult.fail(false); + if (lockRequestForNewSegment.getGranularity() == LockGranularity.SEGMENT) { + newSegmentId = allocateSegmentId(lockRequestForNewSegment, request.getVersion()); + if (newSegmentId == null) { + return LockResult.fail(false); + } + convertedRequest = new SpecificSegmentLockRequest(lockRequestForNewSegment, newSegmentId); + } else { + convertedRequest = new TimeChunkLockRequest(lockRequestForNewSegment); } - convertedRequest = new SpecificSegmentLockRequest(lockRequestForNewSegment, newSegmentId); } else { - newSegmentId = null; convertedRequest = request; } final TaskLockPosse posseToUse = createOrFindLockPosse(convertedRequest); if (posseToUse != null && !posseToUse.getTaskLock().isRevoked()) { + if (request instanceof LockRequestForNewSegment) { + final LockRequestForNewSegment lockRequestForNewSegment = (LockRequestForNewSegment) request; + if (lockRequestForNewSegment.getGranularity() == LockGranularity.TIME_CHUNK) { + if (newSegmentId != null) { + throw new ISE( + "SegmentId must be allocated after getting a timeChunk lock," + + " but we already have [%s] before getting the lock?", + newSegmentId + ); + } + newSegmentId = allocateSegmentId(lockRequestForNewSegment, posseToUse.getTaskLock().getVersion()); + } + } + // Add to existing TaskLockPosse, if necessary if (posseToUse.addTask(task)) { log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock()); @@ -503,7 +520,7 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) } } - private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment request) + private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment request, String version) { return metadataStorageCoordinator.allocatePendingSegment( request.getDataSource(), @@ -511,7 +528,7 @@ private SegmentIdWithShardSpec allocateSegmentId(LockRequestForNewSegment reques request.getPrevisousSegmentId(), request.getInterval(), request.getShardSpecFactory(), - request.getVersion(), + version, request.isSkipSegmentLineageCheck() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java index 5ee851ebc00b..3966df66eb73 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TimeChunkLockRequest.java @@ -50,6 +50,19 @@ public TimeChunkLockRequest( this(lockType, task.getGroupId(), task.getDataSource(), interval, preferredVersion, task.getPriority(), false); } + public TimeChunkLockRequest(LockRequestForNewSegment lockRequestForNewSegment) + { + this( + lockRequestForNewSegment.getType(), + lockRequestForNewSegment.getGroupId(), + lockRequestForNewSegment.getDataSource(), + lockRequestForNewSegment.getInterval(), + lockRequestForNewSegment.getVersion(), + lockRequestForNewSegment.getPriority(), + lockRequestForNewSegment.isRevoked() + ); + } + public TimeChunkLockRequest( TaskLockType lockType, String groupId, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 79f1e0313642..0f3d2ed44469 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -30,6 +30,7 @@ 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.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -76,6 +77,7 @@ public abstract class SeekableStreamIndexTask savedParseExceptions; + protected final LockGranularity lockGranularityToUse; // Lazily initialized, to avoid calling it on the overlord when tasks are instantiated. // See https://github.com/apache/incubator-druid/issues/7724 for issues that can cause. @@ -115,6 +117,9 @@ public SeekableStreamIndexTask( this.authorizerMapper = authorizerMapper; this.rowIngestionMetersFactory = rowIngestionMetersFactory; this.runnerSupplier = Suppliers.memoize(this::createTaskRunner); + this.lockGranularityToUse = getContextValue(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, Tasks.DEFAULT_FORCE_TIME_CHUNK_LOCK) + ? LockGranularity.TIME_CHUNK + : LockGranularity.SEGMENT; } private static String makeTaskId(String dataSource, String type) @@ -233,7 +238,8 @@ public StreamAppenderatorDriver newDriver( sequenceName, previousSegmentId, skipSegmentLineageCheck, - NumberedShardSpecFactory.instance() + NumberedShardSpecFactory.instance(), + lockGranularityToUse ) ), toolbox.getSegmentHandoffNotifierFactory(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 87b7eb1f4b87..484a67ffffc5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -46,13 +46,15 @@ import org.apache.druid.indexer.TaskStatus; 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.CheckPointDataSourceMetadataAction; import org.apache.druid.indexing.common.actions.ResetDataSourceMetadataAction; -import org.apache.druid.indexing.common.actions.SegmentLockAquireAction; +import org.apache.druid.indexing.common.actions.SegmentLockAcquireAction; +import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskUtils; @@ -202,6 +204,8 @@ public enum Status private final List> publishWaitList = new ArrayList<>(); private final List> handOffWaitList = new ArrayList<>(); + private final LockGranularity lockGranularityToUse; + private volatile DateTime startTime; private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread) private volatile TaskToolbox toolbox; @@ -222,7 +226,8 @@ public SeekableStreamIndexTaskRunner( final AuthorizerMapper authorizerMapper, final Optional chatHandlerProvider, final CircularBuffer savedParseExceptions, - final RowIngestionMetersFactory rowIngestionMetersFactory + final RowIngestionMetersFactory rowIngestionMetersFactory, + final LockGranularity lockGranularityToUse ) { Preconditions.checkNotNull(task); @@ -238,6 +243,7 @@ public SeekableStreamIndexTaskRunner( this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); this.sequences = new CopyOnWriteArrayList<>(); this.ingestionState = IngestionState.NOT_STARTED; + this.lockGranularityToUse = lockGranularityToUse; resetNextCheckpointTime(); } @@ -388,15 +394,25 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception final Object restoredMetadata = driver.startJob( segmentId -> { try { - return toolbox.getTaskActionClient().submit( - new SegmentLockAquireAction( - TaskLockType.EXCLUSIVE, - segmentId.getInterval(), - segmentId.getVersion(), - segmentId.getShardSpec().getPartitionNum(), - 1000L - ) - ).isOk(); + if (lockGranularityToUse == 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); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index ba1fc2bf24fe..526862069ee9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -51,6 +51,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.IOException; import java.util.HashMap; @@ -59,6 +61,7 @@ import java.util.Map.Entry; import java.util.stream.Collectors; +@RunWith(Parameterized.class) public class SegmentAllocateActionTest { @Rule @@ -71,6 +74,22 @@ public class SegmentAllocateActionTest private static final DateTime PARTY_TIME = DateTimes.of("1999"); private static final DateTime THE_DISTANT_FUTURE = DateTimes.of("3000"); + private final LockGranularity lockGranularity; + + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.SEGMENT}, + new Object[]{LockGranularity.TIME_CHUNK} + ); + } + + public SegmentAllocateActionTest(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + } + @Before public void setUp() { @@ -711,7 +730,8 @@ public void testSerde() throws Exception "s1", "prev", false, - NumberedShardSpecFactory.instance() + NumberedShardSpecFactory.instance(), + lockGranularity ); final SegmentAllocateAction action2 = (SegmentAllocateAction) objectMapper.readValue( @@ -761,7 +781,8 @@ public void testWithShardSpecFactoryAndOvershadowingSegments() throws IOExceptio "seq", null, true, - new HashBasedNumberedShardSpecFactory(ImmutableList.of("dim1"), 2) + new HashBasedNumberedShardSpecFactory(ImmutableList.of("dim1"), 2), + lockGranularity ); final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox()); Assert.assertNotNull(segmentIdentifier); @@ -813,7 +834,8 @@ private SegmentIdWithShardSpec allocate( sequenceName, sequencePreviousId, false, - shardSpecFactory + shardSpecFactory, + lockGranularity ); return action.perform(task, taskActionTestKit.getTaskActionToolbox()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index f1a9ced3959e..45aaba40a434 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -30,6 +30,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SegmentLoaderFactory; @@ -65,6 +66,8 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.BufferedWriter; @@ -80,6 +83,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +@RunWith(Parameterized.class) public class CompactionTaskRunTest extends IngestionTestBase { public static final String DATA_SOURCE = "test"; @@ -107,13 +111,23 @@ public class CompactionTaskRunTest extends IngestionTestBase 0 ); - private RowIngestionMetersFactory rowIngestionMetersFactory; - private CoordinatorClient coordinatorClient; - private SegmentLoaderFactory segmentLoaderFactory; + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK}, + new Object[]{LockGranularity.SEGMENT} + ); + } + + private static final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig()); + private final RowIngestionMetersFactory rowIngestionMetersFactory; + private final CoordinatorClient coordinatorClient; + private final SegmentLoaderFactory segmentLoaderFactory; + private final LockGranularity lockGranularity; private ExecutorService exec; - private static RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig()); - public CompactionTaskRunTest() + public CompactionTaskRunTest(LockGranularity lockGranularity) { TestUtils testUtils = new TestUtils(); rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory(); @@ -126,6 +140,7 @@ public List getDatabaseSegmentDataSourceSegments(String dataSource, } }; segmentLoaderFactory = new SegmentLoaderFactory(getIndexIO(), getObjectMapper()); + this.lockGranularity = lockGranularity; } @Before @@ -168,8 +183,18 @@ public void testRun() throws Exception Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { - Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1), segments.get(i).getShardSpec()); + Assert.assertEquals( + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), + segments.get(i).getInterval() + ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + } } } @@ -201,11 +226,18 @@ public void testRunCompactionTwice() throws Exception Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { - Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); Assert.assertEquals( - new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), - segments.get(i).getShardSpec() + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), + segments.get(i).getInterval() ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + } } final CompactionTask compactionTask2 = builder @@ -220,11 +252,24 @@ public void testRunCompactionTwice() throws Exception Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { - Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); Assert.assertEquals( - new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, 0, 2, (short) 2, (short) 1), - segments.get(i).getShardSpec() + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), + segments.get(i).getInterval() ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, + 0, + 2, + (short) 2, + (short) 1 + ), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + } } } @@ -308,11 +353,18 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc Assert.assertEquals(3, segments.size()); for (int i = 0; i < 3; i++) { - Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); Assert.assertEquals( - new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), - segments.get(i).getShardSpec() + Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), + segments.get(i).getInterval() ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, 2, (short) 1, (short) 1), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + } } } @@ -451,10 +503,20 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), segments.get(i).getInterval() ); - Assert.assertEquals( - new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), - segments.get(i).getShardSpec() - ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, + 0, + 2, + (short) 1, + (short) 2 + ), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + } } final Pair> compactionResult = compactionFuture.get(); @@ -512,10 +574,20 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i / 2, i / 2 + 1), segments.get(i).getInterval() ); - Assert.assertEquals( - new NumberedOverwriteShardSpec(PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, 0, 2, (short) 1, (short) 2), - segments.get(i).getShardSpec() - ); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + i % 2, + 0, + 2, + (short) 1, + (short) 2 + ), + segments.get(i).getShardSpec() + ); + } else { + Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + } } final Pair> compactionResult = compactionFuture.get(); @@ -649,6 +721,7 @@ public File getPath() new NoopTestTaskFileWriter() ); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); if (task.isReady(box.getTaskActionClient())) { if (readyLatchToCountDown != null) { readyLatchToCountDown.countDown(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java index 12e1cfffbec5..905eee028fbe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopTaskTest.java @@ -70,23 +70,17 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - public boolean requireLockInputSegments() + public boolean requireLockExistingSegments() { return true; } @Override - public List findInputSegments(TaskActionClient taskActionClient, List intervals) + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) { return Collections.emptyList(); } - @Override - public boolean checkIfChangeSegmentGranularity(List intervalOfExistingSegments) - { - return true; - } - @Override public boolean isPerfectRollup() { @@ -95,7 +89,7 @@ public boolean isPerfectRollup() @Nullable @Override - public Granularity getSegmentGranularity(Interval interval) + public Granularity getSegmentGranularity() { return null; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index f7c9cef75167..8df8f404d5e0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -38,6 +38,7 @@ 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.TaskReport; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.stats.RowIngestionMeters; @@ -86,6 +87,8 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.BufferedWriter; @@ -100,6 +103,7 @@ import java.util.Map; import java.util.Set; +@RunWith(Parameterized.class) public class IndexTaskTest extends IngestionTestBase { @Rule @@ -125,19 +129,29 @@ public class IndexTaskTest extends IngestionTestBase 0 ); - private SegmentLoader segmentLoader; + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK}, + new Object[]{LockGranularity.SEGMENT} + ); + } private static final IndexSpec indexSpec = new IndexSpec(); private final ObjectMapper jsonMapper; private final IndexIO indexIO; private final RowIngestionMetersFactory rowIngestionMetersFactory; + private final LockGranularity lockGranularity; + private SegmentLoader segmentLoader; private TestTaskRunner taskRunner; - public IndexTaskTest() + public IndexTaskTest(LockGranularity lockGranularity) { - jsonMapper = getObjectMapper(); - indexIO = getIndexIO(); - rowIngestionMetersFactory = getRowIngestionMetersFactory(); + this.jsonMapper = getObjectMapper(); + this.indexIO = getIndexIO(); + this.rowIngestionMetersFactory = getRowIngestionMetersFactory(); + this.lockGranularity = lockGranularity; } @Before @@ -1483,17 +1497,23 @@ public void testOverwriteWithSameSegmentGranularity() throws Exception Assert.assertEquals(NumberedShardSpec.class, segment.getShardSpec().getClass()); Assert.assertEquals(j, segment.getShardSpec().getPartitionNum()); } else { - Assert.assertEquals(NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass()); - final NumberedOverwriteShardSpec numberedOverwriteShardSpec = - (NumberedOverwriteShardSpec) segment.getShardSpec(); - Assert.assertEquals( - j + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, - numberedOverwriteShardSpec.getPartitionNum() - ); - Assert.assertEquals(1, numberedOverwriteShardSpec.getMinorVersion()); - Assert.assertEquals(5, numberedOverwriteShardSpec.getAtomicUpdateGroupSize()); - Assert.assertEquals(0, numberedOverwriteShardSpec.getStartRootPartitionId()); - Assert.assertEquals(5, numberedOverwriteShardSpec.getEndRootPartitionId()); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals(NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass()); + final NumberedOverwriteShardSpec numberedOverwriteShardSpec = + (NumberedOverwriteShardSpec) segment.getShardSpec(); + Assert.assertEquals( + j + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, + numberedOverwriteShardSpec.getPartitionNum() + ); + Assert.assertEquals(1, numberedOverwriteShardSpec.getMinorVersion()); + Assert.assertEquals(5, numberedOverwriteShardSpec.getAtomicUpdateGroupSize()); + Assert.assertEquals(0, numberedOverwriteShardSpec.getStartRootPartitionId()); + Assert.assertEquals(5, numberedOverwriteShardSpec.getEndRootPartitionId()); + } else { + Assert.assertEquals(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec numberedShardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(j, numberedShardSpec.getPartitionNum()); + } } } } @@ -1556,6 +1576,7 @@ public static void checkTaskStatusErrorMsgForParseExceptionsExceeded(TaskStatus private Pair> runTask(IndexTask task) throws Exception { + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); final TaskStatus status = taskRunner.run(task).get(); final List segments = taskRunner.getPublishedSegments(); return Pair.of(status, segments); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index fd3d127025ce..489ca25b83e5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -19,14 +19,17 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.google.common.collect.ImmutableList; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -44,6 +47,8 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.File; @@ -58,10 +63,26 @@ import java.util.Map; import java.util.stream.Collectors; +@RunWith(Parameterized.class) public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest { + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK}, + new Object[]{LockGranularity.SEGMENT} + ); + } + + private final LockGranularity lockGranularity; private File inputDir; + public ParallelIndexSupervisorTaskTest(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + } + @Before public void setup() throws IOException { @@ -131,32 +152,39 @@ public void testIsReady() throws Exception } } - private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity) throws Exception + private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity, boolean appendToExisting) + throws Exception { final ParallelIndexSupervisorTask task = newTask( interval, segmentGranularity, new ParallelIndexIOConfig( new LocalFirehoseFactory(inputDir, "test_*", null), - false + appendToExisting ) ); actionClient = createActionClient(task); toolbox = createTaskToolbox(task); prepareTaskForLocking(task); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertTrue(task.isReady(actionClient)); Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); shutdownTask(task); } - private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity secondSegmentGranularity) throws Exception + private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity) throws Exception + { + runTestTask(interval, segmentGranularity, false); + } + + private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity secondSegmentGranularity) + throws Exception { // Ingest all data. runTestTask(inputInterval, Granularities.DAY); - // Read the segments for one day. - final Interval interval = Intervals.of("2017-12-24/P5D"); + final Interval interval = inputInterval == null ? Intervals.ETERNITY : inputInterval; final List allSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); // Reingest the same data. Each segment should get replaced by a segment with a newer version. @@ -167,10 +195,10 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s allSegments.addAll(newSegments); final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(allSegments); final List visibles = timeline.lookup(interval) - .stream() - .flatMap(holder -> holder.getObject().stream()) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); + .stream() + .flatMap(holder -> holder.getObject().stream()) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); Assert.assertEquals(newSegments, visibles); } @@ -221,6 +249,7 @@ public boolean isSplittable() toolbox = createTaskToolbox(task); prepareTaskForLocking(task); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertTrue(task.isReady(actionClient)); Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); } @@ -239,6 +268,7 @@ public void testPublishEmptySegments() throws Exception toolbox = createTaskToolbox(task); prepareTaskForLocking(task); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertTrue(task.isReady(actionClient)); Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); } @@ -280,18 +310,38 @@ public void testWith1MaxNumSubTasks() throws Exception toolbox = createTaskToolbox(task); prepareTaskForLocking(task); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertTrue(task.isReady(actionClient)); Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); Assert.assertNull("Runner must be null if the task was in the sequential mode", task.getRunner()); } - private ParallelIndexSupervisorTask newTask(Interval interval, ParallelIndexIOConfig ioConfig) + @Test + public void testAppendToExisting() throws Exception + { + final Interval interval = Intervals.of("2017/2018"); + runTestTask(interval, Granularities.DAY, true); + final List oldSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + + runTestTask(interval, Granularities.DAY, true); + final List newSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + Assert.assertTrue(newSegments.containsAll(oldSegments)); + final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(newSegments); + final List visibles = timeline.lookup(interval) + .stream() + .flatMap(holder -> holder.getObject().stream()) + .map(PartitionChunk::getObject) + .collect(Collectors.toList()); + Assert.assertEquals(newSegments, visibles); + } + + private ParallelIndexSupervisorTask newTask(@Nullable Interval interval, ParallelIndexIOConfig ioConfig) { return newTask(interval, Granularities.DAY, ioConfig); } private ParallelIndexSupervisorTask newTask( - Interval interval, + @Nullable Interval interval, Granularity segmentGranularity, ParallelIndexIOConfig ioConfig ) @@ -326,7 +376,7 @@ private ParallelIndexSupervisorTask newTask( } private ParallelIndexSupervisorTask newTask( - Interval interval, + @Nullable Interval interval, Granularity segmentGranularity, ParallelIndexIOConfig ioConfig, ParallelIndexTuningConfig tuningConfig diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 0111d32a2c62..7290952ac352 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -983,6 +983,7 @@ private void allocateSegmentsAndAssert( { for (int i = 0; i < numSegmentsToAllocate; i++) { final LockRequestForNewSegment request = new LockRequestForNewSegment( + LockGranularity.SEGMENT, TaskLockType.EXCLUSIVE, task, Intervals.of("2015-01-01/2015-01-05"), diff --git a/pom.xml b/pom.xml index 9033ffdd524d..433ef326fcb8 100644 --- a/pom.xml +++ b/pom.xml @@ -1237,7 +1237,7 @@ - -Xmx4g + -Xmx1500m -XX:MaxDirectMemorySize=512m -Duser.language=en -Duser.GroupByQueryRunnerTest.javacountry=US diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index ead6498fbdbe..9eacec56bebe 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -95,6 +95,8 @@ default List getUsedSegmentsForInterval(String dataSource, Interval * @param sequenceName name of the group of ingestion tasks producing a segment series * @param previousSegmentId previous segment in the series; may be null or empty, meaning this is the first segment * @param interval interval for which to allocate a segment + * @param shardSpecFactory shardSpecFactory containing all necessary information to create a shardSpec for the + * new segmentId * @param maxVersion use this version if we have no better version to use. The returned segment identifier may * have a version lower than this one, but will not have one higher. * @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence. diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 7c435968c08f..f6f710293a98 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -667,12 +667,16 @@ private SegmentIdWithShardSpec createNewSegment( return null; } + // max partitionId of the SAME shardSpec SegmentIdWithShardSpec maxId = null; if (!existingChunks.isEmpty()) { TimelineObjectHolder existingHolder = Iterables.getOnlyElement(existingChunks); maxId = StreamSupport.stream(existingHolder.getObject().spliterator(), false) + // Here we check only the segments of the same shardSpec to find out the max partitionId. + // Note that OverwriteShardSpec has the higher range for partitionId than others. + // See PartitionIds. .filter(chunk -> chunk.getObject().getShardSpec().getClass() == shardSpecFactory.getShardSpecClass()) .max(Comparator.comparing(chunk -> chunk.getObject().getShardSpec().getPartitionNum())) .map(chunk -> SegmentIdWithShardSpec.fromDataSegment(chunk.getObject())) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverSegmentLockHelper.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorDriverSegmentLockHelper.java similarity index 89% rename from server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverSegmentLockHelper.java rename to server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorDriverSegmentLockHelper.java index ba1b44cd4db1..2de7089e314d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriverSegmentLockHelper.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorDriverSegmentLockHelper.java @@ -22,7 +22,9 @@ /** * Lock helper for {@link StreamAppenderatorDriver}. It's only used to lock segments on restart of the driver. */ -public interface StreamAppenderatorDriverSegmentLockHelper +public interface AppenderatorDriverSegmentLockHelper { + AppenderatorDriverSegmentLockHelper NOOP = segmentId -> true; + boolean lock(SegmentIdWithShardSpec segmentId); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index 1914659997d8..707bff82007d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -271,7 +271,7 @@ Map getSegments() * @return currently persisted commit metadata */ @Nullable - public abstract Object startJob(StreamAppenderatorDriverSegmentLockHelper lockHelper); + public abstract Object startJob(AppenderatorDriverSegmentLockHelper lockHelper); /** * Find a segment in the {@link SegmentState#APPENDING} state for the given timestamp and sequenceName. diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java index 1264e004cd03..e9e946bbb74b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java @@ -73,6 +73,12 @@ public BatchAppenderatorDriver( super(appenderator, segmentAllocator, usedSegmentChecker, dataSegmentKiller); } + @Nullable + public Object startJob() + { + return startJob(AppenderatorDriverSegmentLockHelper.NOOP); + } + /** * This method always returns null because batch ingestion doesn't support restoring tasks on failures. * @@ -80,7 +86,7 @@ public BatchAppenderatorDriver( */ @Override @Nullable - public Object startJob(StreamAppenderatorDriverSegmentLockHelper lockHelper) + public Object startJob(AppenderatorDriverSegmentLockHelper lockHelper) { final Object metadata = appenderator.startJob(); if (metadata != null) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index 918104fbd2a6..c68b4a9bb270 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -107,7 +107,7 @@ public StreamAppenderatorDriver( @Override @Nullable - public Object startJob(StreamAppenderatorDriverSegmentLockHelper lockHelper) + public Object startJob(AppenderatorDriverSegmentLockHelper lockHelper) { handoffNotifier.start(); From 2bcb0999b5253280f479cb8903749810fd202685 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 3 Jul 2019 22:06:10 -0700 Subject: [PATCH 35/52] fix travis --- .../timeline/partition/PartitionIds.java | 10 +- .../AppenderatorDriverRealtimeIndexTask.java | 35 +- .../batch/parallel/ParallelIndexSubTask.java | 18 +- .../actions/SegmentAllocateActionTest.java | 434 ++++++++++++------ ...rallelIndexSupervisorTaskResourceTest.java | 1 + 5 files changed, 335 insertions(+), 163 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java index 155da827ec3c..9f655856288b 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java @@ -24,21 +24,21 @@ public final class PartitionIds /** * Start partitionId available for root generation segments. */ - public static int ROOT_GEN_START_PARTITION_ID = 0; + public final static int ROOT_GEN_START_PARTITION_ID = 0; /** * End partitionId available for root generation segments. */ - public static int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive + public final static int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive /** * Start partitionId available for non-root generation segments. */ - public static int NON_ROOT_GEN_START_PARTITION_ID = 32768; + public final static int NON_ROOT_GEN_START_PARTITION_ID = 32768; /** * End partitionId available for non-root generation segments. */ - public static int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive + public final static int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive - public static short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; + public final static short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; private PartitionIds() { 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 9c51fa9ad6fa..d271c7028281 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 @@ -53,6 +53,7 @@ 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; @@ -165,6 +166,9 @@ private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec) @JsonIgnore private final AuthorizerMapper authorizerMapper; + @JsonIgnore + private final LockGranularity lockGranularity; + @JsonIgnore private IngestionState ingestionState; @@ -200,6 +204,9 @@ public AppenderatorDriverRealtimeIndexTask( 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 @@ -282,15 +289,25 @@ public TaskStatus run(final TaskToolbox toolbox) driver.startJob( segmentId -> { try { - return toolbox.getTaskActionClient().submit( - new SegmentLockAcquireAction( - TaskLockType.EXCLUSIVE, - segmentId.getInterval(), - segmentId.getVersion(), - segmentId.getShardSpec().getPartitionNum(), - 1000L - ) - ).isOk(); + 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); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index efa7a329bfc0..4d355ea30fcd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -291,25 +291,9 @@ private SegmentAllocator createSegmentAllocator() final Interval interval = granularitySpec .bucketInterval(row.getTimestamp()) .or(granularitySpec.getSegmentGranularity().bucket(row.getTimestamp())); - final Granularity granularityFromInterval = GranularityType.fromPeriod(interval.toPeriod()) - .getDefaultGranularity(); + final ShardSpecFactory shardSpecFactory; if (segmentLockHelper.hasOverwritingRootGenerationPartition(interval)) { - if (segmentLockHelper.getKnownSegmentGranularity() != null - && !segmentLockHelper.getKnownSegmentGranularity().equals(granularityFromInterval)) { - throw new ISE( - "Different segment granularity is detected. knownGranularity[%s], granularityFromInterval[%s]", - segmentLockHelper.getKnownSegmentGranularity(), - granularityFromInterval - ); - } - final boolean lockedExistingSegments = segmentLockHelper.tryLockExistingSegments( - toolbox.getTaskActionClient(), - Collections.singletonList(interval) - ); - if (!lockedExistingSegments) { - throw new ISE("Failed to lock segments in interval[%s]", interval); - } final OverwritingRootGenerationPartitions overwritingSegmentMeta = segmentLockHelper .getOverwritingRootGenerationPartition(interval); if (overwritingSegmentMeta == null) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 526862069ee9..5cabcf598cb6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -20,8 +20,11 @@ package org.apache.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Predicate; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLock; @@ -166,41 +169,76 @@ public void testManySegmentsSameInterval() id2.toString() ); - final List partyTimeLocks = taskActionTestKit.getTaskLockbox() - .findLocksForTask(task) - .stream() - .filter(input -> input.getInterval().contains(PARTY_TIME)) - .collect(Collectors.toList()); + if (lockGranularity == LockGranularity.TIME_CHUNK) { + final TaskLock partyLock = Iterables.getOnlyElement( + FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) + .filter(input -> input.getInterval().contains(PARTY_TIME)) + ); - Assert.assertEquals(3, partyTimeLocks.size()); + assertSameIdentifier( + id1, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLock.getVersion(), + new NumberedShardSpec(0, 0) + ) + ); + assertSameIdentifier( + id2, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLock.getVersion(), + new NumberedShardSpec(1, 0) + ) + ); + assertSameIdentifier( + id3, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLock.getVersion(), + new NumberedShardSpec(2, 0) + ) + ); + } else { + final List partyTimeLocks = taskActionTestKit.getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(PARTY_TIME)) + .collect(Collectors.toList()); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - id1.getVersion(), - new NumberedShardSpec(0, 0) - ), - id1 - ); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - id1.getVersion(), - new NumberedShardSpec(1, 0) - ), - id2 - ); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - id1.getVersion(), - new NumberedShardSpec(2, 0) - ), - id3 - ); + Assert.assertEquals(3, partyTimeLocks.size()); + + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + id1.getVersion(), + new NumberedShardSpec(0, 0) + ), + id1 + ); + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + id1.getVersion(), + new NumberedShardSpec(1, 0) + ), + id2 + ); + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + id1.getVersion(), + new NumberedShardSpec(2, 0) + ), + id3 + ); + } } @Test @@ -281,57 +319,114 @@ public void testResumeSequence() Assert.assertNotNull(id7); allocatedFutureIds.put(id7.getShardSpec().getPartitionNum(), id7); - final List partyLocks = taskActionTestKit.getTaskLockbox() - .findLocksForTask(task) - .stream() - .filter(input -> input.getInterval().contains(PARTY_TIME)) - .collect(Collectors.toList()); - - Assert.assertEquals(2, partyLocks.size()); - final Map partitionIdToLock = new HashMap<>(); - partyLocks.forEach(lock -> { - Assert.assertEquals(LockGranularity.SEGMENT, lock.getGranularity()); - final SegmentLock segmentLock = (SegmentLock) lock; - partitionIdToLock.put(segmentLock.getPartitionId(), segmentLock); - }); - - for (Entry entry : partitionIdToLock.entrySet()) { + if (lockGranularity == LockGranularity.TIME_CHUNK) { + final TaskLock partyLock = Iterables.getOnlyElement( + FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) + .filter( + new Predicate() + { + @Override + public boolean apply(TaskLock input) + { + return input.getInterval().contains(PARTY_TIME); + } + } + ) + ); + final TaskLock futureLock = Iterables.getOnlyElement( + FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) + .filter( + new Predicate() + { + @Override + public boolean apply(TaskLock input) + { + return input.getInterval().contains(THE_DISTANT_FUTURE); + } + } + ) + ); + assertSameIdentifier( + id1, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(PARTY_TIME), - allocatedPartyTimeIds.get(entry.getKey()).getVersion(), - new NumberedShardSpec(entry.getValue().getPartitionId(), 0) - ), - allocatedPartyTimeIds.get(entry.getKey()) + partyLock.getVersion(), + new NumberedShardSpec(0, 0) + ) ); - } - - final List futureLocks = taskActionTestKit - .getTaskLockbox() - .findLocksForTask(task) - .stream() - .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) - .collect(Collectors.toList()); - - Assert.assertEquals(1, futureLocks.size()); - partitionIdToLock.clear(); - futureLocks.forEach(lock -> { - Assert.assertEquals(LockGranularity.SEGMENT, lock.getGranularity()); - final SegmentLock segmentLock = (SegmentLock) lock; - partitionIdToLock.put(segmentLock.getPartitionId(), segmentLock); - }); - - for (Entry entry : partitionIdToLock.entrySet()) { assertSameIdentifier( + id2, new SegmentIdWithShardSpec( DATA_SOURCE, Granularities.HOUR.bucket(THE_DISTANT_FUTURE), - allocatedFutureIds.get(entry.getKey()).getVersion(), - new NumberedShardSpec(entry.getValue().getPartitionId(), 0) - ), - allocatedFutureIds.get(entry.getKey()) + futureLock.getVersion(), + new NumberedShardSpec(0, 0) + ) + ); + assertSameIdentifier( + id3, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLock.getVersion(), + new NumberedShardSpec(1, 0) + ) ); + } else { + final List partyLocks = taskActionTestKit.getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(PARTY_TIME)) + .collect(Collectors.toList()); + + Assert.assertEquals(2, partyLocks.size()); + final Map partitionIdToLock = new HashMap<>(); + partyLocks.forEach(lock -> { + Assert.assertEquals(LockGranularity.SEGMENT, lock.getGranularity()); + final SegmentLock segmentLock = (SegmentLock) lock; + partitionIdToLock.put(segmentLock.getPartitionId(), segmentLock); + }); + + for (Entry entry : partitionIdToLock.entrySet()) { + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + allocatedPartyTimeIds.get(entry.getKey()).getVersion(), + new NumberedShardSpec(entry.getValue().getPartitionId(), 0) + ), + allocatedPartyTimeIds.get(entry.getKey()) + ); + } + + final List futureLocks = taskActionTestKit + .getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) + .collect(Collectors.toList()); + + Assert.assertEquals(1, futureLocks.size()); + partitionIdToLock.clear(); + futureLocks.forEach(lock -> { + Assert.assertEquals(LockGranularity.SEGMENT, lock.getGranularity()); + final SegmentLock segmentLock = (SegmentLock) lock; + partitionIdToLock.put(segmentLock.getPartitionId(), segmentLock); + }); + + for (Entry entry : partitionIdToLock.entrySet()) { + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(THE_DISTANT_FUTURE), + allocatedFutureIds.get(entry.getKey()).getVersion(), + new NumberedShardSpec(entry.getValue().getPartitionId(), 0) + ), + allocatedFutureIds.get(entry.getKey()) + ); + } } Assert.assertNull(id4); @@ -382,69 +477,144 @@ public void testMultipleSequences() null ); - final List partyLocks = taskActionTestKit.getTaskLockbox() - .findLocksForTask(task) - .stream() - .filter(input -> input.getInterval().contains(PARTY_TIME)) - .collect(Collectors.toList()); + if (lockGranularity == LockGranularity.TIME_CHUNK) { + final TaskLock partyLock = Iterables.getOnlyElement( + FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) + .filter( + new Predicate() + { + @Override + public boolean apply(TaskLock input) + { + return input.getInterval().contains(PARTY_TIME); + } + } + ) + ); + final TaskLock futureLock = Iterables.getOnlyElement( + FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task)) + .filter( + new Predicate() + { + @Override + public boolean apply(TaskLock input) + { + return input.getInterval().contains(THE_DISTANT_FUTURE); + } + } + ) + ); - Assert.assertEquals(3, partyLocks.size()); + assertSameIdentifier( + id1, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLock.getVersion(), + new NumberedShardSpec(0, 0) + ) + ); + assertSameIdentifier( + id2, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLock.getVersion(), + new NumberedShardSpec(1, 0) + ) + ); + assertSameIdentifier( + id3, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLock.getVersion(), + new NumberedShardSpec(2, 0) + ) + ); + assertSameIdentifier( + id4, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(THE_DISTANT_FUTURE), + futureLock.getVersion(), + new NumberedShardSpec(0, 0) + ) + ); + assertSameIdentifier( + id5, + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(THE_DISTANT_FUTURE), + futureLock.getVersion(), + new NumberedShardSpec(1, 0) + ) + ); + } else { + final List partyLocks = taskActionTestKit.getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(PARTY_TIME)) + .collect(Collectors.toList()); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - partyLocks.get(0).getVersion(), - new NumberedShardSpec(0, 0) - ), - id1 - ); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - partyLocks.get(1).getVersion(), - new NumberedShardSpec(1, 0) - ), - id2 - ); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(PARTY_TIME), - partyLocks.get(2).getVersion(), - new NumberedShardSpec(2, 0) - ), - id3 - ); + Assert.assertEquals(3, partyLocks.size()); + + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLocks.get(0).getVersion(), + new NumberedShardSpec(0, 0) + ), + id1 + ); + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLocks.get(1).getVersion(), + new NumberedShardSpec(1, 0) + ), + id2 + ); + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(PARTY_TIME), + partyLocks.get(2).getVersion(), + new NumberedShardSpec(2, 0) + ), + id3 + ); - final List futureLocks = taskActionTestKit - .getTaskLockbox() - .findLocksForTask(task) - .stream() - .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) - .collect(Collectors.toList()); + final List futureLocks = taskActionTestKit + .getTaskLockbox() + .findLocksForTask(task) + .stream() + .filter(input -> input.getInterval().contains(THE_DISTANT_FUTURE)) + .collect(Collectors.toList()); - Assert.assertEquals(2, futureLocks.size()); + Assert.assertEquals(2, futureLocks.size()); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(THE_DISTANT_FUTURE), - futureLocks.get(0).getVersion(), - new NumberedShardSpec(0, 0) - ), - id4 - ); - assertSameIdentifier( - new SegmentIdWithShardSpec( - DATA_SOURCE, - Granularities.HOUR.bucket(THE_DISTANT_FUTURE), - futureLocks.get(1).getVersion(), - new NumberedShardSpec(1, 0) - ), - id5 - ); + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(THE_DISTANT_FUTURE), + futureLocks.get(0).getVersion(), + new NumberedShardSpec(0, 0) + ), + id4 + ); + assertSameIdentifier( + new SegmentIdWithShardSpec( + DATA_SOURCE, + Granularities.HOUR.bucket(THE_DISTANT_FUTURE), + futureLocks.get(1).getVersion(), + new NumberedShardSpec(1, 0) + ), + id5 + ); + } assertSameIdentifier(id1, id6); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 15a8fec548db..c508084a9b6f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -499,6 +499,7 @@ private class TestSupervisorTask extends TestParallelIndexSupervisorTask @Override ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) { + setToolbox(toolbox); setRunner( new TestRunner( toolbox, From 4c0d91b594a86123cbcf94a78035a05ccefd5c81 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 3 Jul 2019 22:49:22 -0700 Subject: [PATCH 36/52] fix travis --- .../apache/druid/timeline/partition/PartitionIds.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java index 9f655856288b..fc5d0e981c45 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionIds.java @@ -24,21 +24,21 @@ public final class PartitionIds /** * Start partitionId available for root generation segments. */ - public final static int ROOT_GEN_START_PARTITION_ID = 0; + public static final int ROOT_GEN_START_PARTITION_ID = 0; /** * End partitionId available for root generation segments. */ - public final static int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive + public static final int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive /** * Start partitionId available for non-root generation segments. */ - public final static int NON_ROOT_GEN_START_PARTITION_ID = 32768; + public static final int NON_ROOT_GEN_START_PARTITION_ID = 32768; /** * End partitionId available for non-root generation segments. */ - public final static int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive + public static final int NON_ROOT_GEN_END_PARTITION_ID = 65536; // exclusive - public final static short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; + public static final short UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE = -1; private PartitionIds() { From 27a828b3ca5a938d0960282ced57e8e657424d47 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 3 Jul 2019 23:39:50 -0700 Subject: [PATCH 37/52] unused import --- .../common/task/batch/parallel/ParallelIndexSubTask.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 4d355ea30fcd..047f5a12278c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -50,7 +50,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; 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.java.util.common.parsers.ParseException; import org.apache.druid.query.DruidMetrics; From 5a54070665b248019368eb56c7839922e0af6fa1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 4 Jul 2019 10:49:12 -0700 Subject: [PATCH 38/52] spotbug --- .../main/java/org/apache/druid/server/SegmentManager.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index 6417822b79c9..c43a73bd01b0 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -218,6 +218,7 @@ public void dropSegment(final DataSegment segment) (dataSourceName, dataSourceState) -> { if (dataSourceState == null) { log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSourceName); + return null; } else { final VersionedIntervalTimeline loadedIntervals = dataSourceState.getTimeline(); @@ -253,10 +254,10 @@ public void dropSegment(final DataSegment segment) segment.getMajorVersion() ); } - } - // Returning null removes the entry of dataSource from the map - return dataSourceState == null || dataSourceState.isEmpty() ? null : dataSourceState; + // Returning null removes the entry of dataSource from the map + return dataSourceState.isEmpty() ? null : dataSourceState; + } } ); From 29e4d2a8b9e5d283b24db5cd871f7f0cfd50c7de Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 4 Jul 2019 21:06:03 -0700 Subject: [PATCH 39/52] revert getMaxVersion --- .../NewestSegmentFirstPolicyBenchmark.java | 2 +- .../segment/loading/DataSegmentPusher.java | 4 ++-- .../apache/druid/timeline/DataSegment.java | 4 ++-- .../apache/druid/timeline/Overshadowable.java | 4 ++-- .../timeline/VersionedIntervalTimeline.java | 2 +- .../timeline/partition/AtomicUpdateGroup.java | 4 ++-- .../druid/timeline/DataSegmentTest.java | 2 +- .../SegmentWithOvershadowedStatusTest.java | 2 +- .../VersionedIntervalTimelineTest.java | 2 +- .../partition/IntegerPartitionChunkTest.java | 2 +- .../storage/azure/AzureDataSegmentPusher.java | 2 +- .../MaterializedViewSupervisor.java | 2 +- .../storage/hdfs/HdfsDataSegmentPusher.java | 2 +- .../druid/indexer/IndexGeneratorJobTest.java | 2 +- .../indexing/common/actions/TaskLocks.java | 8 ++++---- .../indexing/common/task/ArchiveTask.java | 4 ++-- .../druid/indexing/common/task/MoveTask.java | 4 ++-- .../indexing/common/task/RestoreTask.java | 4 ++-- .../common/task/SegmentLockHelper.java | 2 +- .../IngestSegmentFirehoseFactory.java | 4 ++-- ...penderatorDriverRealtimeIndexTaskTest.java | 18 ++++++++--------- .../common/task/RealtimeIndexTaskTest.java | 10 +++++----- .../segment/ReferenceCountingSegment.java | 2 +- .../druid/segment/SchemalessIndexTest.java | 2 +- .../apache/druid/client/BrokerServerView.java | 6 +++--- .../druid/client/CoordinatorServerView.java | 6 +++--- .../druid/client/DataSourcesSnapshot.java | 4 ++-- .../apache/druid/client/SegmentLoadInfo.java | 4 ++-- .../druid/client/selector/ServerSelector.java | 4 ++-- .../SQLMetadataStorageUpdaterJobHandler.java | 2 +- .../IndexerSQLMetadataStorageCoordinator.java | 2 +- .../metadata/SQLMetadataSegmentManager.java | 2 +- .../metadata/SQLMetadataSegmentPublisher.java | 2 +- .../appenderator/AppenderatorImpl.java | 6 +++--- .../appenderator/SegmentIdWithShardSpec.java | 2 +- ...oordinatorBasedSegmentHandoffNotifier.java | 2 +- .../realtime/plumber/RealtimePlumber.java | 6 +++--- .../druid/segment/realtime/plumber/Sink.java | 2 +- .../apache/druid/server/SegmentManager.java | 8 ++++---- .../DruidCoordinatorRuntimeParams.java | 2 +- .../DruidCoordinatorCleanupOvershadowed.java | 2 +- .../server/http/DataSourcesResource.java | 2 +- .../client/CachingClusteredClientTest.java | 4 ++-- .../SQLMetadataSegmentManagerTest.java | 2 +- .../appenderator/TestUsedSegmentChecker.java | 2 +- .../druid/server/SegmentManagerTest.java | 2 +- .../DruidCoordinatorSegmentCompactorTest.java | 20 +++++++++---------- .../server/shard/NumberedShardSpecTest.java | 2 +- .../org/apache/druid/cli/ExportMetadata.java | 2 +- .../sql/calcite/schema/SystemSchema.java | 2 +- .../SpecificSegmentsQuerySegmentWalker.java | 2 +- 51 files changed, 98 insertions(+), 98 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index c19824cf7b31..d06125ac35df 100644 --- a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -127,7 +127,7 @@ public void setup() 0, segmentSizeBytes ); - timeline.add(segment.getInterval(), segment.getMajorVersion(), shardSpec.createChunk(segment)); + timeline.add(segment.getInterval(), segment.getVersion(), shardSpec.createChunk(segment)); } } diff --git a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java index e0e610732b8a..7e9eb9234b76 100644 --- a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java +++ b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java @@ -108,7 +108,7 @@ static String getDefaultStorageDir(DataSegment segment, boolean useUniquePath) return JOINER.join( segment.getDataSource(), StringUtils.format("%s_%s", segment.getInterval().getStart(), segment.getInterval().getEnd()), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec().getPartitionNum(), useUniquePath ? generateUniquePath() : null ); @@ -119,7 +119,7 @@ static String getDefaultStorageDirWithExistingUniquePath(DataSegment segment, St return JOINER.join( segment.getDataSource(), StringUtils.format("%s_%s", segment.getInterval().getStart(), segment.getInterval().getEnd()), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec().getPartitionNum(), uniquePath ); 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 fe573cfac4b3..a8e2fe77f42a 100644 --- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -231,7 +231,7 @@ public Map getLoadSpec() @JsonProperty("version") @Override - public String getMajorVersion() + public String getVersion() { return id.getVersion(); } @@ -452,7 +452,7 @@ public Builder(DataSegment segment) { this.dataSource = segment.getDataSource(); this.interval = segment.getInterval(); - this.version = segment.getMajorVersion(); + this.version = segment.getVersion(); this.loadSpec = segment.getLoadSpec(); this.dimensions = segment.getDimensions(); this.metrics = segment.getMetrics(); diff --git a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java index 562e8661a8e9..238314e88665 100644 --- a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java +++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java @@ -34,7 +34,7 @@ public interface Overshadowable */ default boolean isOvershadow(T other) { - final int majorVersionCompare = getMajorVersion().compareTo(other.getMajorVersion()); + final int majorVersionCompare = getVersion().compareTo(other.getVersion()); if (majorVersionCompare == 0) { return containsRootPartition(other) && getMinorVersion() > other.getMinorVersion(); } else { @@ -65,7 +65,7 @@ default boolean containsRootPartition(T other) */ int getEndRootPartitionId(); - String getMajorVersion(); + String getVersion(); short getMinorVersion(); 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 761ec5e6397a..0734379809ea 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -114,7 +114,7 @@ public static void addSegments( timeline.addAll( Iterators.transform(segments, segment -> segment.getShardSpec().createChunk(segment)), DataSegment::getInterval, - DataSegment::getMajorVersion + DataSegment::getVersion ); } 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 index b200a4d896cd..e72e96b41af9 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -126,10 +126,10 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { Preconditions.checkState(!isEmpty(), "Empty atomicUpdateGroup"); - return chunks.get(0).getObject().getMajorVersion(); + return chunks.get(0).getObject().getVersion(); } @Override 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 ef3d692c4904..bc3cf902b9ed 100644 --- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -149,7 +149,7 @@ public void testV1Serialization() throws Exception Assert.assertEquals(segment.getDataSource(), deserializedSegment.getDataSource()); Assert.assertEquals(segment.getInterval(), deserializedSegment.getInterval()); - Assert.assertEquals(segment.getMajorVersion(), deserializedSegment.getMajorVersion()); + Assert.assertEquals(segment.getVersion(), deserializedSegment.getVersion()); Assert.assertEquals(segment.getLoadSpec(), deserializedSegment.getLoadSpec()); Assert.assertEquals(segment.getDimensions(), deserializedSegment.getDimensions()); Assert.assertEquals(segment.getMetrics(), deserializedSegment.getMetrics()); diff --git a/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java b/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java index 06a1a90728d9..050f9e04934b 100644 --- a/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java +++ b/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java @@ -100,7 +100,7 @@ public void testUnwrappedSegmentWithOvershadowedStatusDeserialization() throws E Assert.assertEquals(segment.getDataSegment().getDataSource(), deserializedSegment.getDataSource()); Assert.assertEquals(segment.getDataSegment().getInterval(), deserializedSegment.getInterval()); - Assert.assertEquals(segment.getDataSegment().getMajorVersion(), deserializedSegment.getMajorVersion()); + Assert.assertEquals(segment.getDataSegment().getVersion(), deserializedSegment.getVersion()); Assert.assertEquals(segment.getDataSegment().getLoadSpec(), deserializedSegment.getLoadSpec()); Assert.assertEquals(segment.getDataSegment().getDimensions(), deserializedSegment.getDimensions()); Assert.assertEquals(segment.getDataSegment().getMetrics(), deserializedSegment.getMetrics()); 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 2d89ce25b7dc..03d217cc08c0 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -2240,7 +2240,7 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { return majorVersion; } 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 04d9a36ca70b..f2ac7a32db82 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 @@ -111,7 +111,7 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { return ""; } diff --git a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java index ad6ad9eebfb7..2d2a8bc81b20 100644 --- a/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java +++ b/extensions-contrib/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureDataSegmentPusher.java @@ -89,7 +89,7 @@ public String getStorageDir(DataSegment dataSegment, boolean useUniquePath) dataSegment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()), dataSegment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime()) ), - dataSegment.getMajorVersion().replace(':', '_'), + dataSegment.getVersion().replace(':', '_'), dataSegment.getShardSpec().getPartitionNum(), useUniquePath ? DataSegmentPusher.generateUniquePath() : null ); diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index ff64e22e2c25..76883ea3f891 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -431,7 +431,7 @@ private Pair, Map>> getVersion Map> segments = new HashMap<>(); for (DataSegment segment : snapshot) { Interval interval = segment.getInterval(); - versions.put(interval, segment.getMajorVersion()); + versions.put(interval, segment.getVersion()); segments.putIfAbsent(interval, new ArrayList<>()); segments.get(interval).add(segment); } diff --git a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java index 76d39c3fa1ed..fd479768530c 100644 --- a/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java +++ b/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java @@ -203,7 +203,7 @@ public String getStorageDir(DataSegment segment, boolean useUniquePath) segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()), segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime()) ), - segment.getMajorVersion().replace(':', '_') + segment.getVersion().replace(':', '_') ); } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index 2bde82b7ad19..3925bcd730c9 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -651,7 +651,7 @@ private void verifyJob(IndexGeneratorJob job) throws IOException final DataSegment dataSegment = segments.get(i); final File indexZip = indexFiles.get(i); - Assert.assertEquals(config.getSchema().getTuningConfig().getVersion(), dataSegment.getMajorVersion()); + Assert.assertEquals(config.getSchema().getTuningConfig().getVersion(), dataSegment.getVersion()); Assert.assertEquals("local", dataSegment.getLoadSpec().get("type")); Assert.assertEquals(indexZip.getCanonicalPath(), dataSegment.getLoadSpec().get("path")); Assert.assertEquals(Integer.valueOf(9), dataSegment.getBinaryVersion()); 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 index 1e0847903a3b..a7feb4fc5a2a 100644 --- 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 @@ -96,12 +96,12 @@ public static boolean isLockCoversSegments( final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; return timeChunkLock.getInterval().contains(segment.getInterval()) && timeChunkLock.getDataSource().equals(segment.getDataSource()) - && timeChunkLock.getVersion().compareTo(segment.getMajorVersion()) >= 0; + && 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.getMajorVersion()) >= 0 + && segmentLock.getVersion().compareTo(segment.getVersion()) >= 0 && segmentLock.getPartitionId() == segment.getShardSpec().getPartitionNum(); } } @@ -134,14 +134,14 @@ public static List findLocksForSegments( final TimeChunkLock timeChunkLock = (TimeChunkLock) lock; if (timeChunkLock.getInterval().contains(segment.getInterval()) && timeChunkLock.getDataSource().equals(segment.getDataSource()) - && timeChunkLock.getVersion().compareTo(segment.getMajorVersion()) >= 0) { + && 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.getMajorVersion()) >= 0 + && segmentLock.getVersion().compareTo(segment.getVersion()) >= 0 && segmentLock.getPartitionId() == segment.getShardSpec().getPartitionNum()) { found.add(lock); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java index 8f03c658d428..01d2448a88db 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java @@ -81,11 +81,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { - if (unusedSegment.getMajorVersion().compareTo(myLock.getVersion()) > 0) { + if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { throw new ISE( "WTF?! Unused segment[%s] has version[%s] > task version[%s]", unusedSegment.getId(), - unusedSegment.getMajorVersion(), + unusedSegment.getVersion(), myLock.getVersion() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java index 8f3cb08195f4..6bdaef816c1a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java @@ -89,11 +89,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { - if (unusedSegment.getMajorVersion().compareTo(myLock.getVersion()) > 0) { + if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { throw new ISE( "WTF?! Unused segment[%s] has version[%s] > task version[%s]", unusedSegment.getId(), - unusedSegment.getMajorVersion(), + unusedSegment.getVersion(), myLock.getVersion() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java index 8ee87a673721..f3e98ae40289 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java @@ -82,11 +82,11 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { - if (unusedSegment.getMajorVersion().compareTo(myLock.getVersion()) > 0) { + if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) { throw new ISE( "WTF?! Unused segment[%s] has version[%s] > task version[%s]", unusedSegment.getId(), - unusedSegment.getMajorVersion(), + unusedSegment.getVersion(), myLock.getVersion() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java index c59e5af66a9d..679e38b7dfbc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java @@ -197,7 +197,7 @@ private boolean tryLockSegmentsIfNeeded(TaskActionClient actionClient, List> getTimelineForSegmentIds for (Interval interval : windowedSegmentId.getIntervals()) { final TimelineObjectHolder existingHolder = timeline.get(interval); if (existingHolder != null) { - if (!existingHolder.getVersion().equals(segment.getMajorVersion())) { + if (!existingHolder.getVersion().equals(segment.getVersion())) { throw new ISE("Timeline segments with the same interval should have the same version: " + "existing version[%s] vs new segment[%s]", existingHolder.getVersion(), segment); } @@ -362,7 +362,7 @@ private List> getTimelineForSegmentIds timeline.put(interval, new TimelineObjectHolder<>( interval, segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), new PartitionHolder(segment.getShardSpec().createChunk(segment)) )); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 2042d1202bef..2231048d9681 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -380,7 +380,7 @@ public void testBasics() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -442,7 +442,7 @@ public void testLateData() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -507,7 +507,7 @@ public void testMaxRowsPerSegment() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -574,7 +574,7 @@ public void testMaxTotalRows() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -648,7 +648,7 @@ public void testTransformSpec() throws Exception Pair executorRunnablePair = handOffCallbacks.get( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ) ); @@ -778,7 +778,7 @@ public void testNoReportParseExceptions() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -881,7 +881,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -1077,7 +1077,7 @@ public void testRestore() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -1160,7 +1160,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 6ab857f21400..ab9bb6805b88 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -278,7 +278,7 @@ public void testBasics() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -354,7 +354,7 @@ public void testTransformSpec() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -470,7 +470,7 @@ public void testNoReportParseExceptions() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -563,7 +563,7 @@ public void testRestore() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() @@ -658,7 +658,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception Assert.assertEquals( new SegmentDescriptor( publishedSegment.getInterval(), - publishedSegment.getMajorVersion(), + publishedSegment.getVersion(), publishedSegment.getShardSpec().getPartitionNum() ), entry.getKey() diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java index 2cded45e6e4f..da834495c593 100644 --- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java @@ -213,7 +213,7 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { return baseSegment.getId().getVersion(); } diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java index 7c8db0decf65..fb13fd886571 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java @@ -602,7 +602,7 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { return majorVersion; } diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 91e34da2f67a..7df034a0b6e9 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -241,7 +241,7 @@ private void serverAddedSegment(final DruidServerMetadata server, final DataSegm timelines.put(segment.getDataSource(), timeline); } - timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(selector)); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); selectors.put(segmentId, selector); } @@ -284,14 +284,14 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen selectors.remove(segmentId); final PartitionChunk removedPartition = timeline.remove( - segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(selector) + segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector) ); if (removedPartition == null) { log.warn( "Asked to remove timeline entry[interval: %s, version: %s] that doesn't exist", segment.getInterval(), - segment.getMajorVersion() + segment.getVersion() ); } else { runTimelineCallbacks(callback -> callback.segmentRemoved(segment)); diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java index fba090c68609..1f1d801d9e36 100644 --- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java +++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java @@ -147,7 +147,7 @@ private void serverAddedSegment(final DruidServerMetadata server, final DataSegm timeline.add( segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec().createChunk(segmentLoadInfo) ); segmentLoadInfos.put(segmentId, segmentLoadInfo); @@ -174,7 +174,7 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen segmentLoadInfos.remove(segmentId); final PartitionChunk removedPartition = timeline.remove( - segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk( + segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk( new SegmentLoadInfo( segment ) @@ -185,7 +185,7 @@ private void serverRemovedSegment(DruidServerMetadata server, DataSegment segmen log.warn( "Asked to remove timeline entry[interval: %s, version: %s] that doesn't exist", segment.getInterval(), - segment.getMajorVersion() + segment.getVersion() ); } } diff --git a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java index b5a6597f4479..2c31c9e82d89 100644 --- a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java +++ b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java @@ -96,7 +96,7 @@ private List determineOvershadowedSegments() final Map> timelines = new HashMap<>(); segments.forEach(segment -> timelines .computeIfAbsent(segment.getDataSource(), dataSource -> new VersionedIntervalTimeline<>(Ordering.natural())) - .add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment))); + .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment))); // It's fine to add all overshadowed segments to a single collection because only // a small fraction of the segments in the cluster are expected to be overshadowed, @@ -105,7 +105,7 @@ private List determineOvershadowedSegments() for (DataSegment dataSegment : segments) { final VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); if (timeline != null - && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getMajorVersion(), dataSegment)) { + && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion(), dataSegment)) { overshadowedSegments.add(dataSegment.getId()); } } diff --git a/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java b/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java index ec4f172ac094..ef1fbfead1c8 100644 --- a/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java +++ b/server/src/main/java/org/apache/druid/client/SegmentLoadInfo.java @@ -115,9 +115,9 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { - return segment.getMajorVersion(); + return segment.getVersion(); } @Override diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java index 8414a77e92be..be693242b013 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelector.java @@ -192,9 +192,9 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { - return segment.get().getMajorVersion(); + return segment.get().getVersion(); } @Override diff --git a/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java b/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java index cb8e68f01293..71c2cadb105b 100644 --- a/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java +++ b/server/src/main/java/org/apache/druid/indexer/SQLMetadataStorageUpdaterJobHandler.java @@ -74,7 +74,7 @@ public Void withHandle(Handle handle) throws Exception .put("start", segment.getInterval().getStart().toString()) .put("end", segment.getInterval().getEnd().toString()) .put("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .put("version", segment.getMajorVersion()) + .put("version", segment.getVersion()) .put("used", true) .put("payload", mapper.writeValueAsBytes(segment)) .build() diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index f6f710293a98..44e2196ce301 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -792,7 +792,7 @@ private boolean announceHistoricalSegment( .bind("start", segment.getInterval().getStart().toString()) .bind("end", segment.getInterval().getEnd().toString()) .bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) - .bind("version", segment.getMajorVersion()) + .bind("version", segment.getVersion()) .bind("used", used) .bind("payload", jsonMapper.writeValueAsBytes(segment)) .execute(); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index 1c16529b865c..959a99cd9999 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -402,7 +402,7 @@ private int enableSegments( .stream() .filter(segment -> !versionedIntervalTimeline.isOvershadowed( segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), segment )) .forEach(segment -> batch.add( diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java index 9f155679de72..6c33396de69f 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentPublisher.java @@ -71,7 +71,7 @@ public void publishSegment(final DataSegment segment) throws IOException segment.getInterval().getStart().toString(), segment.getInterval().getEnd().toString(), (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, - segment.getMajorVersion(), + segment.getVersion(), true, jsonMapper.writeValueAsBytes(segment) ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 7fc464a323a9..4f7d6bb9b146 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -405,7 +405,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) sinks.put(identifier, retVal); metrics.setSinkCount(sinks.size()); - sinkTimeline.add(retVal.getInterval(), retVal.getMajorVersion(), identifier.getShardSpec().createChunk(retVal)); + sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), identifier.getShardSpec().createChunk(retVal)); } return retVal; @@ -1069,7 +1069,7 @@ private Object bootstrapSinksFromDisk() sinks.put(identifier, currSink); sinkTimeline.add( currSink.getInterval(), - currSink.getMajorVersion(), + currSink.getVersion(), identifier.getShardSpec().createChunk(currSink) ); @@ -1163,7 +1163,7 @@ public Void apply(@Nullable Object input) droppingSinks.remove(identifier); sinkTimeline.remove( sink.getInterval(), - sink.getMajorVersion(), + sink.getVersion(), identifier.getShardSpec().createChunk(sink) ); for (FireHydrant hydrant : sink) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java index a786763b5769..6e85794e69d1 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpec.java @@ -134,7 +134,7 @@ public static SegmentIdWithShardSpec fromDataSegment(final DataSegment segment) return new SegmentIdWithShardSpec( segment.getDataSource(), segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec() ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java index 122d0fe4f9c3..bbee720a88ce 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java @@ -141,7 +141,7 @@ static boolean isHandOffComplete(List serverView, Segm if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval()) && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() - && segmentLoadInfo.getSegment().getMajorVersion().compareTo(descriptor.getVersion()) >= 0 + && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 && segmentLoadInfo.getServers().stream().anyMatch(DruidServerMetadata::segmentReplicatable)) { return true; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index 6385dafcf473..005f167a1830 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -488,7 +488,7 @@ public void doRun() } ); handoffNotifier.registerSegmentHandoffCallback( - new SegmentDescriptor(sink.getInterval(), sink.getMajorVersion(), config.getShardSpec().getPartitionNum()), + new SegmentDescriptor(sink.getInterval(), sink.getVersion(), config.getShardSpec().getPartitionNum()), mergeExecutor, new Runnable() { @Override @@ -736,7 +736,7 @@ private void addSink(final Sink sink) metrics.setSinkCount(sinks.size()); sinkTimeline.add( sink.getInterval(), - sink.getMajorVersion(), + sink.getVersion(), new SingleElementPartitionChunk<>(sink) ); try { @@ -880,7 +880,7 @@ protected void abandonSegment(final long truncatedTime, final Sink sink) metrics.setSinkCount(sinks.size()); sinkTimeline.remove( sink.getInterval(), - sink.getMajorVersion(), + sink.getVersion(), new SingleElementPartitionChunk<>(sink) ); for (FireHydrant hydrant : sink) { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index cbfdbe5ea872..cd54bf440428 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -426,7 +426,7 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { return version; } diff --git a/server/src/main/java/org/apache/druid/server/SegmentManager.java b/server/src/main/java/org/apache/druid/server/SegmentManager.java index c43a73bd01b0..3ac10790addb 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -168,7 +168,7 @@ public boolean loadSegment(final DataSegment segment) throws SegmentLoadingExcep dataSourceState.getTimeline(); final PartitionHolder entry = loadedIntervals.findEntry( segment.getInterval(), - segment.getMajorVersion() + segment.getVersion() ); if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { @@ -178,7 +178,7 @@ public boolean loadSegment(final DataSegment segment) throws SegmentLoadingExcep final ReferenceCountingSegment referenceCountingSegment = new ReferenceCountingSegment(adapter); loadedIntervals.add( segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec().createChunk(referenceCountingSegment) ); dataSourceState.addSegment(segment); @@ -226,7 +226,7 @@ public void dropSegment(final DataSegment segment) final ShardSpec shardSpec = segment.getShardSpec(); final PartitionChunk removed = loadedIntervals.remove( segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), // remove() internally searches for a partitionChunk to remove which is *equal* to the given // partitionChunk. Note that partitionChunk.equals() checks only the partitionNum, but not the object. segment.getShardSpec().createChunk( @@ -251,7 +251,7 @@ public void dropSegment(final DataSegment segment) "Told to delete a queryable on dataSource[%s] for interval[%s] and version[%s] that I don't have.", dataSourceName, segment.getInterval(), - segment.getMajorVersion() + segment.getVersion() ); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index 4709ccc8a0e2..de75bce9d9c2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -362,7 +362,7 @@ public Builder withDataSources(Collection dataSourcesC dataSource.getSegments().forEach( segment -> timeline.add( segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec().createChunk(segment) ) ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index 3fdac3a30805..660ec7029ae6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -73,7 +73,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) //Remove all segments in db that are overshadowed by served segments for (DataSegment dataSegment : params.getAvailableSegments()) { VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getMajorVersion(), dataSegment)) { + if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion(), dataSegment)) { coordinator.removeSegment(dataSegment); stats.addToGlobalStat("overShadowedCount", 1); } diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 9f2aef4a9d4f..9255e7c51450 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -742,7 +742,7 @@ static boolean isSegmentLoaded(Iterable serverView, Se for (ImmutableSegmentLoadInfo segmentLoadInfo : serverView) { if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval()) && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() - && segmentLoadInfo.getSegment().getMajorVersion().compareTo(descriptor.getVersion()) >= 0 + && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 && Iterables.any( segmentLoadInfo.getServers(), DruidServerMetadata::segmentReplicatable )) { diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 987638b53513..f6346145bfc6 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2827,7 +2827,7 @@ public Map getLoadSpec() @Override @JsonProperty - public String getMajorVersion() + public String getVersion() { return "version"; } @@ -2935,7 +2935,7 @@ public boolean isOvershadow(DataSegment other) { if (getDataSource().equals(other.getDataSource()) && getInterval().overlaps(other.getInterval()) - && getMajorVersion().equals(other.getMajorVersion())) { + && getVersion().equals(other.getVersion())) { return getStartRootPartitionId() <= other.getStartRootPartitionId() && getEndRootPartitionId() >= other.getEndRootPartitionId() && getMinorVersion() > other.getMinorVersion(); diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java index 114ce07da001..a5d436b226e3 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java @@ -99,7 +99,7 @@ private void publish(DataSegment segment, boolean used) throws IOException segment.getInterval().getStart().toString(), segment.getInterval().getEnd().toString(), (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, - segment.getMajorVersion(), + segment.getVersion(), used, jsonMapper.writeValueAsBytes(segment) ); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java index 44a32f9f1609..4d7d1108dd91 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java @@ -44,7 +44,7 @@ public Set findUsedSegments(Set identifiers for (DataSegment dataSegment : appenderatorTester.getPushedSegments()) { timeline.add( dataSegment.getInterval(), - dataSegment.getMajorVersion(), + dataSegment.getVersion(), dataSegment.getShardSpec().createChunk(dataSegment) ); } diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java index e6152b2798e8..14cbdbd90a4c 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -402,7 +402,7 @@ private void assertResult(List expectedExistingSegments) throws Seg ); expectedTimeline.add( segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec().createChunk(new ReferenceCountingSegment(segmentLoader.getSegment(segment))) ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index 1e0c605a1111..bdf711fa6153 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -93,13 +93,13 @@ public String compactSegments( segments.forEach( segment -> timeline.remove( segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), segment.getShardSpec().createChunk(segment) ) ); timeline.add( compactInterval, - compactSegment.getMajorVersion(), + compactSegment.getVersion(), compactSegment.getShardSpec().createChunk(compactSegment) ); return "task_" + idSuffix++; @@ -134,18 +134,18 @@ public void setup() for (int j = 0; j < 4; j++) { for (int k = 0; k < 2; k++) { DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); } } for (int j = 7; j < 9; j++) { for (int k = 0; k < 2; k++) { DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getMajorVersion(), segment.getShardSpec().createChunk(segment)); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); } } @@ -315,7 +315,7 @@ private void assertCompactSegments( Assert.assertEquals(1, chunks.size()); DataSegment segment = chunks.get(0).getObject(); Assert.assertEquals(expectedInterval, segment.getInterval()); - Assert.assertEquals(expectedVersionSupplier.get(), segment.getMajorVersion()); + Assert.assertEquals(expectedVersionSupplier.get(), segment.getVersion()); } } @@ -333,7 +333,7 @@ private void assertLastSegmentNotCompacted(DruidCoordinatorSegmentCompactor comp for (PartitionChunk chunk : chunks) { DataSegment segment = chunk.getObject(); Assert.assertEquals(interval, segment.getInterval()); - Assert.assertEquals("version", segment.getMajorVersion()); + Assert.assertEquals("version", segment.getVersion()); } } } @@ -363,13 +363,13 @@ private void addMoreData(String dataSource, int day) DataSegment newSegment = createSegment(dataSource, day, true, i); dataSources.get(dataSource).add( newSegment.getInterval(), - newSegment.getMajorVersion(), + newSegment.getVersion(), newSegment.getShardSpec().createChunk(newSegment) ); newSegment = createSegment(dataSource, day, false, i); dataSources.get(dataSource).add( newSegment.getInterval(), - newSegment.getMajorVersion(), + newSegment.getVersion(), newSegment.getShardSpec().createChunk(newSegment) ); } diff --git a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java index 90110695de93..10bca175be38 100644 --- a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java @@ -277,7 +277,7 @@ public int getEndRootPartitionId() } @Override - public String getMajorVersion() + public String getVersion() { return "v1"; } diff --git a/services/src/main/java/org/apache/druid/cli/ExportMetadata.java b/services/src/main/java/org/apache/druid/cli/ExportMetadata.java index 6b94263285cb..27f5509570e4 100644 --- a/services/src/main/java/org/apache/druid/cli/ExportMetadata.java +++ b/services/src/main/java/org/apache/druid/cli/ExportMetadata.java @@ -442,7 +442,7 @@ private String makePayloadWithConvertedLoadSpec( segment = new DataSegment( segment.getDataSource(), segment.getInterval(), - segment.getMajorVersion(), + segment.getVersion(), newLoadSpec, segment.getDimensions(), segment.getMetrics(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 9de22f6e9728..42b7e949c880 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -298,7 +298,7 @@ public Enumerable scan(DataContext root) segment.getInterval().getStart().toString(), segment.getInterval().getEnd().toString(), segment.getSize(), - segment.getMajorVersion(), + segment.getVersion(), Long.valueOf(segment.getShardSpec().getPartitionNum()), numReplicas, numRows, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 62c72aceefba..e17fc4544f64 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -82,7 +82,7 @@ public SpecificSegmentsQuerySegmentWalker add( timelines.computeIfAbsent(descriptor.getDataSource(), datasource -> new VersionedIntervalTimeline<>(Ordering.natural())); final VersionedIntervalTimeline timeline = timelines.get(descriptor.getDataSource()); - timeline.add(descriptor.getInterval(), descriptor.getMajorVersion(), descriptor.getShardSpec().createChunk(new ReferenceCountingSegment(segment))); + timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(new ReferenceCountingSegment(segment))); segments.add(descriptor); closeables.add(index); return this; From 82abc45451395ffdcd63afa3070a854d20cccf81 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 4 Jul 2019 21:53:48 -0700 Subject: [PATCH 40/52] address comments --- .../apache/druid/timeline/Overshadowable.java | 2 +- .../druid/indexing/common/SegmentLock.java | 21 +-- .../common/task/AbstractBatchIndexTask.java | 127 +++++++++------- .../task/CachingLocalSegmentAllocator.java | 2 +- .../indexing/common/task/HadoopIndexTask.java | 9 +- .../druid/indexing/common/task/IndexTask.java | 8 +- .../common/task/SegmentLockHelper.java | 142 ++++++------------ .../batch/parallel/ParallelIndexSubTask.java | 12 +- .../parallel/ParallelIndexSupervisorTask.java | 8 +- 9 files changed, 159 insertions(+), 172 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java index 238314e88665..b051229d1681 100644 --- a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java +++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java @@ -30,7 +30,7 @@ public interface Overshadowable { /** - * Returns true this overshadows the given other. + * Returns true if this segment overshadows the other segment. */ default boolean isOvershadow(T other) { 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 index a2edfc6abeb5..be903e0a4a0f 100644 --- 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 @@ -171,24 +171,27 @@ public boolean isRevoked() @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 dataSource.equals(request.getDataSource()) - && interval.overlaps(request.getInterval()); + return interval.overlaps(request.getInterval()); } else if (request instanceof SpecificSegmentLockRequest) { - if (dataSource.equals(request.getDataSource()) - && interval.equals(request.getInterval())) { + 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 dataSource.equals(request.getDataSource()) - && interval.overlaps(request.getInterval()); + return interval.overlaps(request.getInterval()); } - } else if (request instanceof LockRequestForNewSegment) { - // request for new segments doens't conflict with any locks because it allocates a new partitionId - return false; } else { throw new ISE("Unknown request type[%s]", request.getClass().getCanonicalName()); } 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 index 166549e975c5..9f8ab3abac21 100644 --- 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 @@ -63,12 +63,17 @@ 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 context) { super(id, dataSource, context); - segmentLockHelper = new SegmentLockHelper(dataSource); + segmentLockHelper = new SegmentLockHelper(); } protected AbstractBatchIndexTask( @@ -80,7 +85,7 @@ protected AbstractBatchIndexTask( ) { super(id, groupId, taskResource, dataSource, context); - segmentLockHelper = new SegmentLockHelper(dataSource); + segmentLockHelper = new SegmentLockHelper(); } /** @@ -97,17 +102,33 @@ protected AbstractBatchIndexTask( public abstract List findSegmentsToLock(TaskActionClient taskActionClient, List 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; } - @Nullable - public abstract Granularity getSegmentGranularity(); + public SegmentLockHelper getSegmentLockHelper() + { + return segmentLockHelper; + } - public boolean determineLockGranularityAndTryLock( + /** + * 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 @@ -118,18 +139,13 @@ public boolean determineLockGranularityAndTryLock( return determineLockGranularityandTryLock(client, intervals); } - public SegmentLockHelper getNonNullSegmentLockHelper() - { - return Preconditions.checkNotNull(segmentLockHelper, "segmentLockHelper"); - } - - protected boolean determineLockGranularityandTryLock(TaskActionClient client, List intervals) - throws IOException + boolean determineLockGranularityandTryLock(TaskActionClient client, List 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; @@ -149,7 +165,7 @@ protected boolean determineLockGranularityandTryLock(TaskActionClient client, Li } } - protected boolean determineLockGranularityandTryLockWithSegments(TaskActionClient client, List segments) + boolean determineLockGranularityandTryLockWithSegments(TaskActionClient client, List segments) throws IOException { final boolean forceTimeChunkLock = getContextValue( @@ -170,24 +186,8 @@ protected boolean determineLockGranularityandTryLockWithSegments(TaskActionClien } } - private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranularityDeterminResult result) + private LockGranularityDeterminResult determineSegmentGranularity(TaskActionClient client, List intervals) throws IOException - { - if (result.lockGranularity == LockGranularity.TIME_CHUNK) { - return tryTimeChunkLock(client, Preconditions.checkNotNull(result.intervals, "intervals")); - } else { - final boolean isReady = segmentLockHelper.verifyAndLockExistingSegments( - client, - Preconditions.checkNotNull(result.segments, "segments") - ); - return isReady; - } - } - - protected LockGranularityDeterminResult determineSegmentGranularity( - TaskActionClient client, - List intervals - ) throws IOException { if (requireLockExistingSegments()) { if (isPerfectRollup()) { @@ -211,6 +211,19 @@ protected LockGranularityDeterminResult determineSegmentGranularity( } } + private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranularityDeterminResult 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 intervals) throws IOException { // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined @@ -233,24 +246,7 @@ private boolean tryTimeChunkLock(TaskActionClient client, List interva return true; } - @Nullable - public static Granularity findGranularityFromSegments(List 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; - } - } - - LockGranularityDeterminResult determineSegmentGranularity(List segments) + private LockGranularityDeterminResult determineSegmentGranularity(List segments) { if (segments.isEmpty()) { log.info("Using segment lock for empty segments"); @@ -300,14 +296,41 @@ LockGranularityDeterminResult determineSegmentGranularity(List segm } } + @Nullable + static Granularity findGranularityFromSegments(List 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 findInputSegments( String dataSource, TaskActionClient actionClient, - List intervalsToFind, // TODO: must be checked somewhere? probably? + List intervalsToRead, FirehoseFactory firehoseFactory ) throws IOException { if (firehoseFactory instanceof IngestSegmentFirehoseFactory) { + // intervalsToRead is ignored here. final List inputSegments = ((IngestSegmentFirehoseFactory) firehoseFactory).getSegments(); if (inputSegments == null) { final Interval inputInterval = Preconditions.checkNotNull( @@ -336,7 +359,7 @@ protected static List findInputSegments( .collect(Collectors.toList()); } } else { - return actionClient.submit(new SegmentListUsedAction(dataSource, null, intervalsToFind)); + return actionClient.submit(new SegmentListUsedAction(dataSource, null, intervalsToRead)); } } @@ -344,9 +367,9 @@ private static class LockGranularityDeterminResult { private final LockGranularity lockGranularity; @Nullable - private final List intervals; + private final List intervals; // null for segmentLock @Nullable - private final List segments; + private final List segments; // null for timeChunkLock private LockGranularityDeterminResult( LockGranularity lockGranularity, 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 index 0fa6f8791b32..8803f9db041c 100644 --- 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 @@ -44,7 +44,7 @@ import java.util.stream.IntStream; /** - * Allocates all necessary segments at the beginning and reuse them. + * Allocates all necessary segments locally at the beginning and reuse them. */ class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator { 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 9daab19fcdeb..f8728c0c0e77 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 @@ -56,6 +56,8 @@ 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; @@ -227,7 +229,12 @@ public boolean isPerfectRollup() @Override public Granularity getSegmentGranularity() { - return spec.getDataSchema().getGranularitySpec().getSegmentGranularity(); + final GranularitySpec granularitySpec = spec.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } } @JsonProperty("spec") 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 8cf019a29036..884d61e649ec 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 @@ -561,7 +561,7 @@ private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningCon * them. If the perfect rollup must be guaranteed, {@link HashBasedNumberedShardSpec} is used for hash partitioning * of input data. In the future we may want to also support single-dimension partitioning. * - * @return generated {@link ShardSpecs} representing a map of intervals and corresponding shard specs + * @return a map indicating how many shardSpecs need to be created per interval. */ private Map> determineShardSpecs( final TaskToolbox toolbox, @@ -800,7 +800,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( toolbox, getId(), dataSchema, - getNonNullSegmentLockHelper(), + getSegmentLockHelper(), isUseSegmentLock() ? LockGranularity.SEGMENT : LockGranularity.TIME_CHUNK, ingestionSchema.ioConfig.isAppendToExisting() ); @@ -831,7 +831,7 @@ private IndexTaskSegmentAllocator createSegmentAllocator( *

* At the end of this method, all the remaining segments are published. * - * @return true if generated segments are successfully published, otherwise false + * @return the last {@link TaskStatus} */ private TaskStatus generateAndPublishSegments( final TaskToolbox toolbox, @@ -940,7 +940,7 @@ private TaskStatus generateAndPublishSegments( // Probably we can publish atomicUpdateGroup along with segments. final Set inputSegments = isUseSegmentLock() - ? getNonNullSegmentLockHelper().getLockedExistingSegments() + ? getSegmentLockHelper().getLockedExistingSegments() : null; final SegmentsAndMetadata published = awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java index 679e38b7dfbc..020f29efa3ae 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java @@ -22,16 +22,12 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexing.common.SegmentLock; import org.apache.druid.indexing.common.TaskLockType; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; import org.apache.druid.indexing.common.actions.SegmentLockReleaseAction; import org.apache.druid.indexing.common.actions.SegmentLockTryAcquireAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; -import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.java.util.common.ISE; 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.io.Closer; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -44,12 +40,15 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.stream.Collectors; +/** + * This class provides some methods to use the segment lock easier and caches the information of locked segments. + */ public class SegmentLockHelper { - private final String dataSource; private final Map overwritingRootGenPartitions = new HashMap<>(); private final Set lockedExistingSegments = new HashSet<>(); @Nullable @@ -84,11 +83,6 @@ public short getMinorVersionForNewSegments() } } - public SegmentLockHelper(String dataSource) - { - this.dataSource = dataSource; - } - public boolean hasLockedExistingSegments() { return !lockedExistingSegments.isEmpty(); @@ -109,36 +103,7 @@ public OverwritingRootGenerationPartitions getOverwritingRootGenerationPartition return overwritingRootGenPartitions.get(interval); } - @Nullable - public Granularity getKnownSegmentGranularity() - { - return knownSegmentGranularity; - } - - public boolean tryLockExistingSegments( - TaskActionClient actionClient, - List intervalsToFind // intervals must be aligned with the segment granularity of existing segments - ) throws IOException - { - final List segments = actionClient.submit( - new SegmentListUsedAction( - dataSource, - null, - intervalsToFind - ) - ); - return verifyAndLockExistingSegments(actionClient, segments); - } - - public boolean tryLockExistingSegments( - TaskActionClient actionClient, - IngestSegmentFirehoseFactory firehoseFactory - ) throws IOException - { - return verifyAndLockExistingSegments(actionClient, findExistingSegments(actionClient, firehoseFactory)); - } - - public boolean verifyAndLockExistingSegments(TaskActionClient actionClient, List segments) + boolean verifyAndLockExistingSegments(TaskActionClient actionClient, List segments) throws IOException { final List segmentsToLock = segments.stream() @@ -149,9 +114,12 @@ public boolean verifyAndLockExistingSegments(TaskActionClient actionClient, List } verifySegmentGranularity(segmentsToLock); - return tryLockSegmentsIfNeeded(actionClient, segmentsToLock); + return tryLockSegments(actionClient, segmentsToLock); } + /** + * Check if segmentGranularity has changed. + */ private void verifySegmentGranularity(List segments) { final Granularity granularityFromSegments = AbstractBatchIndexTask.findGranularityFromSegments(segments); @@ -187,31 +155,43 @@ private void verifySegmentGranularity(List segments) } } - private boolean tryLockSegmentsIfNeeded(TaskActionClient actionClient, List segments) throws IOException + private boolean tryLockSegments(TaskActionClient actionClient, List segments) throws IOException { final Map> intervalToSegments = groupSegmentsByInterval(segments); final Closer lockCloserOnError = Closer.create(); - for (List segmentsInInterval : intervalToSegments.values()) { - for (DataSegment segment : segmentsInInterval) { - final List lockResults = actionClient.submit( - new SegmentLockTryAcquireAction( - TaskLockType.EXCLUSIVE, - segment.getInterval(), - segment.getVersion(), Collections.singleton(segment.getShardSpec().getPartitionNum()) - ) - ); - lockResults.stream() - .filter(LockResult::isOk) - .map(result -> (SegmentLock) result.getTaskLock()) - .forEach(segmentLock -> lockCloserOnError.register(() -> actionClient.submit( - new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) - ))); - if (lockResults.stream().anyMatch(result -> !result.isOk())) { - lockCloserOnError.close(); - return false; - } - lockedExistingSegments.add(segment); + for (Entry> entry : intervalToSegments.entrySet()) { + final Interval interval = entry.getKey(); + final List segmentsInInterval = entry.getValue(); + final boolean hasSameVersion = segmentsInInterval + .stream() + .allMatch(segment -> segment.getVersion().equals(segmentsInInterval.get(0).getVersion())); + Preconditions.checkState( + hasSameVersion, + "Segments[%s] should have same version", + segmentsInInterval.stream().map(DataSegment::getId).collect(Collectors.toList()) + ); + final List lockResults = actionClient.submit( + new SegmentLockTryAcquireAction( + TaskLockType.EXCLUSIVE, + interval, + segmentsInInterval.get(0).getVersion(), + segmentsInInterval.stream() + .map(segment -> segment.getShardSpec().getPartitionNum()) + .collect(Collectors.toSet()) + ) + ); + + lockResults.stream() + .filter(LockResult::isOk) + .map(result -> (SegmentLock) result.getTaskLock()) + .forEach(segmentLock -> lockCloserOnError.register(() -> actionClient.submit( + new SegmentLockReleaseAction(segmentLock.getInterval(), segmentLock.getPartitionId()) + ))); + if (lockResults.stream().anyMatch(result -> !result.isOk())) { + lockCloserOnError.close(); + return false; } + lockedExistingSegments.addAll(segmentsInInterval); verifyAndFindRootPartitionRangeAndMinorVersion(segmentsInInterval); } return true; @@ -300,39 +280,6 @@ public static void verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(List< } } - private List findExistingSegments( - TaskActionClient actionClient, - IngestSegmentFirehoseFactory firehoseFactory - ) throws IOException - { - final List inputSegments = firehoseFactory.getSegments(); - if (inputSegments == null) { - final Interval inputInterval = Preconditions.checkNotNull(firehoseFactory.getInterval(), "input interval"); - - return actionClient.submit( - new SegmentListUsedAction(dataSource, null, Collections.singletonList(inputInterval)) - ); - } else { - final List inputSegmentIds = inputSegments.stream() - .map(WindowedSegmentId::getSegmentId) - .collect(Collectors.toList()); - final Set dataSegmentsInIntervals = new HashSet<>( - 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()); - } - } - private static Map> groupSegmentsByInterval(List segments) { final Map> map = new HashMap<>(); @@ -341,9 +288,4 @@ private static Map> groupSegmentsByInterval(List allSegments = new HashSet<>(getNonNullSegmentLockHelper().getLockedExistingSegments()); + final Set allSegments = new HashSet<>(getSegmentLockHelper().getLockedExistingSegments()); allSegments.addAll(pushedSegments); final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(allSegments); final Set oldSegments = timeline.findFullyOvershadowed() @@ -242,7 +243,12 @@ public boolean isPerfectRollup() @Override public Granularity getSegmentGranularity() { - return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } } @VisibleForTesting @@ -281,7 +287,7 @@ public SegmentIdWithShardSpec allocate( private SegmentAllocator createSegmentAllocator() { final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); - final SegmentLockHelper segmentLockHelper = getNonNullSegmentLockHelper(); + final SegmentLockHelper segmentLockHelper = getSegmentLockHelper(); if (ingestionSchema.getIOConfig().isAppendToExisting() || isUseSegmentLock()) { return new ActionBasedSegmentAllocator( toolbox.getTaskActionClient(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 3067841136ab..27328f4ad32f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -53,6 +53,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; 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.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -252,7 +253,12 @@ public boolean isPerfectRollup() @Override public Granularity getSegmentGranularity() { - return ingestionSchema.getDataSchema().getGranularitySpec().getSegmentGranularity(); + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } } @Override From c51b44ab833661142797e51466456a194ff7c7f3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 4 Jul 2019 21:54:52 -0700 Subject: [PATCH 41/52] fix tc --- .../indexing/appenderator/SegmentAllocateActionGenerator.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java index 80a0344d8446..07d3f30d4c5a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/SegmentAllocateActionGenerator.java @@ -24,8 +24,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import java.io.IOException; - /** * This class is used in {@link ActionBasedSegmentAllocator} and expected to generate a * {@link TaskAction< SegmentIdWithShardSpec >} which is submitted to overlords to allocate a new segment. @@ -42,5 +40,5 @@ TaskAction generate( String sequenceName, String previousSegmentId, boolean skipSegmentLineageCheck - ) throws IOException; + ); } From 789849cf0037ddb9653ca3d48780e8c5e897f883 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 7 Jul 2019 17:08:33 -0700 Subject: [PATCH 42/52] add missing error handling --- .../apache/druid/indexing/overlord/TaskLockbox.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 3ca7173938ad..0837f0337ba8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -478,9 +478,16 @@ private TaskLockPosse createOrFindLockPosse(LockRequest request) } } } - } else { - // case 2) we found a lock posse for the given task + } else if (reusablePosses.size() == 1) { + // case 2) we found a lock posse for the given request return reusablePosses.get(0); + } else { + // case 3) we found multiple lock posses for the given task + throw new ISE( + "Task group[%s] has multiple locks for the same interval[%s]?", + request.getGroupId(), + request.getInterval() + ); } } else { // We don't have any locks for dataSource and interval. @@ -1051,7 +1058,7 @@ boolean reusableFor(LockRequest request) if (taskLock.getLockType() == request.getType() && taskLock.getGranularity() == request.getGranularity()) { switch (taskLock.getLockType()) { case SHARED: - // All shared lock is not reusable. Instead, a new lock posse is created for all lock request. + // All shared lock is not reusable. Instead, a new lock posse is created for each lock request. // See createOrFindLockPosse(). return false; case EXCLUSIVE: From 320b9c9f7b6d28d3ee028d53d003f06e6274e43d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 9 Jul 2019 14:58:51 -0700 Subject: [PATCH 43/52] fix backward compatibility --- .../druid/indexing/common/SegmentLock.java | 11 +- .../druid/indexing/common/TaskLock.java | 6 +- .../druid/indexing/common/TimeChunkLock.java | 11 +- .../druid/indexing/overlord/TaskLockbox.java | 16 +-- .../druid/indexing/common/TaskLockTest.java | 110 +++++++++++++++++- .../indexing/overlord/TaskLockboxTest.java | 8 +- 6 files changed, 125 insertions(+), 37 deletions(-) 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 index be903e0a4a0f..980c68cfd4d0 100644 --- 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 @@ -49,7 +49,7 @@ public class SegmentLock implements TaskLock @JsonCreator public SegmentLock( - @JsonProperty("lockType") TaskLockType lockType, + @JsonProperty("type") TaskLockType lockType, @JsonProperty("groupId") String groupId, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval, @@ -82,13 +82,6 @@ public SegmentLock( this(lockType, groupId, dataSource, interval, version, partitionId, priority, false); } - @JsonProperty - @Override - public String getType() - { - return TYPE; - } - @Override public TaskLock revokedCopy() { @@ -109,7 +102,7 @@ public LockGranularity getGranularity() @JsonProperty @Override - public TaskLockType getLockType() + public TaskLockType getType() { return lockType; } 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 df8881bc4f92..8c1e5d256033 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 @@ -31,22 +31,20 @@ /** * Represents a lock held by some task. Immutable. */ -@JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = TimeChunkLock.class) +@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 { - String getType(); - TaskLock revokedCopy(); TaskLock withPriority(int priority); LockGranularity getGranularity(); - TaskLockType getLockType(); + TaskLockType getType(); String getGroupId(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java index 871e9d4f9bf0..15bb3ffceb2d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TimeChunkLock.java @@ -44,7 +44,7 @@ public class TimeChunkLock implements TaskLock @JsonCreator public TimeChunkLock( - @JsonProperty("lockType") @Nullable TaskLockType lockType, // nullable for backward compatibility + @JsonProperty("type") @Nullable TaskLockType lockType, // nullable for backward compatibility @JsonProperty("groupId") String groupId, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval, @@ -75,13 +75,6 @@ public TimeChunkLock( this(type, groupId, dataSource, interval, version, priority, false); } - @JsonProperty - @Override - public String getType() - { - return TYPE; - } - @Override public TaskLock revokedCopy() { @@ -118,7 +111,7 @@ public LockGranularity getGranularity() @Override @JsonProperty - public TaskLockType getLockType() + public TaskLockType getType() { return lockType; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 0837f0337ba8..e064ac689513 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -239,7 +239,7 @@ private TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLoc case SEGMENT: final SegmentLock segmentLock = (SegmentLock) taskLock; request = new SpecificSegmentLockRequest( - segmentLock.getLockType(), + segmentLock.getType(), segmentLock.getGroupId(), segmentLock.getDataSource(), segmentLock.getInterval(), @@ -252,7 +252,7 @@ private TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLoc case TIME_CHUNK: final TimeChunkLock timeChunkLock = (TimeChunkLock) taskLock; request = new TimeChunkLockRequest( - timeChunkLock.getLockType(), + timeChunkLock.getType(), timeChunkLock.getGroupId(), timeChunkLock.getDataSource(), timeChunkLock.getInterval(), @@ -578,7 +578,7 @@ private boolean isTaskLocksValid(Task task, List intervals) final List lockPosses = getOnlyTaskLockPosseContainingInterval(task, interval); // Tasks cannot enter the critical section with a shared lock return lockPosses.stream().map(TaskLockPosse::getTaskLock).allMatch( - lock -> !lock.isRevoked() && lock.getLockType() != TaskLockType.SHARED + lock -> !lock.isRevoked() && lock.getType() != TaskLockType.SHARED ); }); } @@ -971,7 +971,7 @@ Map>>> ge private static boolean isAllSharedLocks(List lockPosses) { return lockPosses.stream() - .allMatch(taskLockPosse -> taskLockPosse.getTaskLock().getLockType().equals(TaskLockType.SHARED)); + .allMatch(taskLockPosse -> taskLockPosse.getTaskLock().getType().equals(TaskLockType.SHARED)); } private static boolean isAllRevocable(List lockPosses, int tryLockPriority) @@ -1017,7 +1017,7 @@ TaskLock getTaskLock() boolean addTask(Task task) { - if (taskLock.getLockType() == TaskLockType.EXCLUSIVE) { + if (taskLock.getType() == TaskLockType.EXCLUSIVE) { Preconditions.checkArgument( taskLock.getGroupId().equals(task.getGroupId()), "groupId[%s] of task[%s] is different from the existing lockPosse's groupId[%s]", @@ -1055,8 +1055,8 @@ boolean isTasksEmpty() boolean reusableFor(LockRequest request) { - if (taskLock.getLockType() == request.getType() && taskLock.getGranularity() == request.getGranularity()) { - switch (taskLock.getLockType()) { + if (taskLock.getType() == request.getType() && taskLock.getGranularity() == request.getGranularity()) { + switch (taskLock.getType()) { case SHARED: // All shared lock is not reusable. Instead, a new lock posse is created for each lock request. // See createOrFindLockPosse(). @@ -1076,7 +1076,7 @@ boolean reusableFor(LockRequest request) } //noinspection SuspiciousIndentAfterControlStatement default: - throw new ISE("Unknown lock type[%s]", taskLock.getLockType()); + throw new ISE("Unknown lock type[%s]", taskLock.getType()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java index c611689ada5c..2bfa8ec8bd91 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java @@ -19,12 +19,18 @@ package org.apache.druid.indexing.common; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; import java.io.IOException; public class TaskLockTest @@ -61,14 +67,13 @@ public void testDeserializeTimeChunkLockWithoutType() throws IOException ); final String json = "{\n" - + " \"lockType\" : \"EXCLUSIVE\",\n" + + " \"type\" : \"EXCLUSIVE\",\n" + " \"groupId\" : \"groupId\",\n" + " \"dataSource\" : \"dataSource\",\n" + " \"interval\" : \"2019-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z\",\n" + " \"version\" : \"version\",\n" + " \"priority\" : 100,\n" - + " \"revoked\" : false,\n" - + " \"type\" : \"timeChunk\"\n" + + " \"revoked\" : false\n" + "}"; Assert.assertEquals(expected, objectMapper.readValue(json, TaskLock.class)); @@ -91,4 +96,103 @@ public void testSerdeSegmentLock() throws IOException Assert.assertEquals(lock, fromJson); } + + @Test + public void testSerdeOldLock() throws IOException + { + final OldTaskLock oldTaskLock = new OldTaskLock( + TaskLockType.EXCLUSIVE, + "groupId", + "dataSource", + Intervals.of(("2019/2020")), + "version", + 10, + true + ); + final byte[] json = objectMapper.writeValueAsBytes(oldTaskLock); + final TaskLock fromJson = objectMapper.readValue(json, TaskLock.class); + Assert.assertEquals(LockGranularity.TIME_CHUNK, fromJson.getGranularity()); + Assert.assertEquals(TaskLockType.EXCLUSIVE, fromJson.getType()); + Assert.assertEquals("groupId", fromJson.getGroupId()); + Assert.assertEquals("dataSource", fromJson.getDataSource()); + Assert.assertEquals(Intervals.of("2019/2020"), fromJson.getInterval()); + Assert.assertEquals("version", fromJson.getVersion()); + Assert.assertEquals(10, fromJson.getPriority().intValue()); + Assert.assertTrue(fromJson.isRevoked()); + } + + private static class OldTaskLock + { + 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; + + @JsonCreator + public OldTaskLock( + @JsonProperty("type") @Nullable TaskLockType type, + @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; + } + + @JsonProperty + public TaskLockType getType() + { + return type; + } + + @JsonProperty + public String getGroupId() + { + return groupId; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public String getVersion() + { + return version; + } + + @JsonProperty + @Nullable + public Integer getPriority() + { + return priority; + } + + @JsonProperty + public boolean isRevoked() + { + return revoked; + } + + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 7290952ac352..004c15b3527e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -740,7 +740,7 @@ public void testSegmentLock() throws InterruptedException Assert.assertNull(lockResult.getNewSegmentId()); Assert.assertTrue(lockResult.getTaskLock() instanceof SegmentLock); final SegmentLock segmentLock = (SegmentLock) lockResult.getTaskLock(); - Assert.assertEquals(TaskLockType.EXCLUSIVE, segmentLock.getLockType()); + Assert.assertEquals(TaskLockType.EXCLUSIVE, segmentLock.getType()); Assert.assertEquals(task.getGroupId(), segmentLock.getGroupId()); Assert.assertEquals(task.getDataSource(), segmentLock.getDataSource()); Assert.assertEquals(Intervals.of("2015-01-01/2015-01-02"), segmentLock.getInterval()); @@ -1008,7 +1008,7 @@ private void assertAllocatedSegments( final SegmentLock segmentLock = (SegmentLock) result.getTaskLock(); final SegmentIdWithShardSpec segmentId = result.getNewSegmentId(); - Assert.assertEquals(lockRequest.getType(), segmentLock.getLockType()); + Assert.assertEquals(lockRequest.getType(), segmentLock.getType()); Assert.assertEquals(lockRequest.getGroupId(), segmentLock.getGroupId()); Assert.assertEquals(lockRequest.getDataSource(), segmentLock.getDataSource()); Assert.assertEquals(lockRequest.getInterval(), segmentLock.getInterval()); @@ -1140,9 +1140,9 @@ private static class IntervalLockWithoutPriority extends TimeChunkLock @Override @JsonProperty - public TaskLockType getLockType() + public TaskLockType getType() { - return super.getLockType(); + return super.getType(); } @Override From 4171d003b3936190d6a6246bdfe6b185e7c596db Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 9 Jul 2019 16:28:05 -0700 Subject: [PATCH 44/52] unused import --- .../test/java/org/apache/druid/indexing/common/TaskLockTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java index 2bfa8ec8bd91..24600488fa99 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskLockTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import org.apache.druid.jackson.DefaultObjectMapper; From f1e3653c0fc18fd894fb17006d5ca80e485fdbcc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 15 Jul 2019 15:21:47 -0700 Subject: [PATCH 45/52] Fix perf of versionedIntervalTimeline --- .../timeline/VersionedIntervalTimeline.java | 17 +++-- .../partition/OvershadowableManager.java | 70 +++++++++++-------- .../timeline/partition/PartitionHolder.java | 3 +- 3 files changed, 54 insertions(+), 36 deletions(-) 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 0734379809ea..7b203a663a41 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -355,12 +355,13 @@ public Set> findFullyOvershadowed( // 2. Visible timelineEntries can also have overshadowed segments. Add them to the result too. for (TimelineEntry entry : incompletePartitionsTimeline.values()) { - if (!entry.partitionHolder.getOvershadowed().isEmpty()) { + final List> entryOvershadowed = entry.partitionHolder.getOvershadowed(); + if (!entryOvershadowed.isEmpty()) { retVal.add( new TimelineObjectHolder<>( entry.trueInterval, entry.version, - new PartitionHolder<>(new ArrayList<>(entry.partitionHolder.getOvershadowed())) + new PartitionHolder<>(entryOvershadowed) ) ); } @@ -380,8 +381,16 @@ public boolean isOvershadowed(Interval interval, VersionType version, ObjectType TimelineEntry entry = completePartitionsTimeline.get(interval); if (entry != null) { final int majorVersionCompare = versionComparator.compare(version, entry.getVersion()); - return majorVersionCompare < 0 - || (majorVersionCompare == 0 && entry.partitionHolder.stream().anyMatch(chunk -> chunk.getObject().isOvershadow(object))); + if (majorVersionCompare == 0) { + for (PartitionChunk chunk : entry.partitionHolder) { + if (chunk.getObject().isOvershadow(object)) { + return true; + } + } + return false; + } else { + return majorVersionCompare < 0; + } } Interval lower = completePartitionsTimeline.floorKey( 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 index 72f6a38ec5e2..59ec3273b831 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -35,12 +35,12 @@ 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.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -49,7 +49,6 @@ import java.util.NoSuchElementException; import java.util.Objects; import java.util.TreeMap; -import java.util.stream.Collectors; /** * OvershadowableManager manages the state of {@link AtomicUpdateGroup}. See the below {@link State} for details of @@ -120,7 +119,7 @@ private Short2ObjectSortedMap> createMinorVersionToAugMap(S } } - private void transitPartitionChunkState(AtomicUpdateGroup atomicUpdateGroup, State from, State to) + private void transitAtomicUpdateGroupState(AtomicUpdateGroup atomicUpdateGroup, State from, State to) { Preconditions.checkNotNull(atomicUpdateGroup, "atomicUpdateGroup"); Preconditions.checkArgument(!atomicUpdateGroup.isEmpty(), "empty atomicUpdateGroup"); @@ -168,7 +167,12 @@ private AtomicUpdateGroup tryRemoveChunkFromGroupWithState(PartitionChunk } } - determineVisibleGroupAfterRemove(atomicUpdateGroup, RootPartitionRange.of(chunk), chunk.getObject().getMinorVersion(), state); + determineVisibleGroupAfterRemove( + atomicUpdateGroup, + RootPartitionRange.of(chunk), + chunk.getObject().getMinorVersion(), + state + ); return atomicUpdateGroup; } } @@ -228,10 +232,10 @@ private void transitionStandbyGroupIfFull(AtomicUpdateGroup aug, State newSta // 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 -> transitPartitionChunkState(entry.getValue(), State.VISIBLE, State.OVERSHADOWED)); + .forEach(entry -> transitAtomicUpdateGroupState(entry.getValue(), State.VISIBLE, State.OVERSHADOWED)); findOvershadowedBy(aug, State.STANDBY) - .forEach(entry -> transitPartitionChunkState(entry.getValue(), State.STANDBY, State.OVERSHADOWED)); - transitPartitionChunkState(aug, State.STANDBY, State.VISIBLE); + .forEach(entry -> transitAtomicUpdateGroupState(entry.getValue(), State.STANDBY, State.OVERSHADOWED)); + transitAtomicUpdateGroupState(aug, State.STANDBY, State.VISIBLE); } } } @@ -343,9 +347,9 @@ private void determineVisibleGroupAfterRemove( // Move the atomicUpdateGroup to standby // and move the fully available overshadowed atomicUpdateGroup to visible if (!augOfRemovedChunk.isEmpty()) { - transitPartitionChunkState(augOfRemovedChunk, State.VISIBLE, State.STANDBY); + transitAtomicUpdateGroupState(augOfRemovedChunk, State.VISIBLE, State.STANDBY); } - latestFullAugs.forEach(group -> transitPartitionChunkState(group, State.OVERSHADOWED, State.VISIBLE)); + latestFullAugs.forEach(group -> transitAtomicUpdateGroupState(group, State.OVERSHADOWED, State.VISIBLE)); } } } @@ -365,15 +369,17 @@ private List> findLatestFullyAvailableOvershadowedAtomicUpd } final OvershadowableManager manager = new OvershadowableManager<>(); - overshadowedGroups.stream() - .flatMap(entry -> entry.getValue().getChunks().stream()) - .forEach(manager::addChunk); - - return manager.visibleGroup - .values() - .stream() - .flatMap(versionToGroup -> versionToGroup.values().stream()) - .collect(Collectors.toList()); + for (Short2ObjectMap.Entry> entry : overshadowedGroups) { + for (PartitionChunk chunk : entry.getValue().getChunks()) { + manager.addChunk(chunk); + } + } + + final List> visibles = new ArrayList<>(); + for (Short2ObjectSortedMap> map : manager.visibleGroup.values()) { + visibles.addAll(map.values()); + } + return visibles; } private void removeFrom(AtomicUpdateGroup aug, State state) @@ -466,20 +472,24 @@ public PartitionChunk getChunk(int partitionId) public List> getVisibles() { - return visibleGroup.values() - .stream() - .flatMap(treeMap -> treeMap.values().stream()) - .flatMap(aug -> aug.getChunks().stream()) - .collect(Collectors.toList()); + final List> visibles = new ArrayList<>(); + for (Short2ObjectSortedMap> treeMap : visibleGroup.values()) { + for (AtomicUpdateGroup aug : treeMap.values()) { + visibles.addAll(aug.getChunks()); + } + } + return visibles; } - public Collection> getOvershadowed() + public List> getOvershadowed() { - return overshadowedGroups.values() - .stream() - .flatMap(treeMap -> treeMap.values().stream()) - .flatMap(aug -> aug.getChunks().stream()) - .collect(Collectors.toList()); + final List> overshadowed = new ArrayList<>(); + for (Short2ObjectSortedMap> treeMap : overshadowedGroups.values()) { + for (AtomicUpdateGroup aug : treeMap.values()) { + overshadowed.addAll(aug.getChunks()); + } + } + return overshadowed; } @Override @@ -608,7 +618,7 @@ public Short2ObjectSortedMap subMap(short fromKey, short toKey) if (fromKey <= key && toKey > key) { return this; } else { - throw new IllegalArgumentException(); + throw new IAE("fromKey: %s, toKey: %s, key: %s", fromKey, toKey, key); } } 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 ed2761014340..5aff27f3c271 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 @@ -22,7 +22,6 @@ import com.google.common.collect.Iterables; import org.apache.druid.timeline.Overshadowable; -import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Objects; @@ -126,7 +125,7 @@ public Stream> stream() return StreamSupport.stream(spliterator(), false); } - public Collection> getOvershadowed() + public List> getOvershadowed() { return overshadowableManager.getOvershadowed(); } From e445db46f1822e8b967c32aa2b694eaa8c7ac574 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 18 Jul 2019 00:27:55 -0700 Subject: [PATCH 46/52] fix timeline --- .../timeline/partition/AtomicUpdateGroup.java | 31 +++++-------- .../partition/OvershadowableManager.java | 44 ++++++++----------- .../timeline/partition/PartitionHolder.java | 4 +- .../VersionedIntervalTimelineTest.java | 2 +- 4 files changed, 31 insertions(+), 50 deletions(-) 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 index e72e96b41af9..aed7f15695f1 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/AtomicUpdateGroup.java @@ -52,16 +52,18 @@ public AtomicUpdateGroup(PartitionChunk chunk) public void add(PartitionChunk chunk) { - final PartitionChunk existing = replaceChunkWith(chunk); - if (existing == null) { - if (isFull()) { - throw new IAE("Can't add more chunk[%s] to atomicUpdateGroup[%s]", chunk, chunks); - } - if (!isSameAtomicUpdateGroup(chunks.get(0), chunk)) { - throw new IAE("Can't add chunk[%s] to a different atomicUpdateGroup[%s]", chunk, chunks); + 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 existing : chunks) { + if (existing.equals(chunk)) { + return; } - chunks.add(chunk); } + chunks.add(chunk); } public void remove(PartitionChunk chunk) @@ -98,19 +100,6 @@ public PartitionChunk findChunk(int partitionId) return chunks.stream().filter(chunk -> chunk.getChunkNumber() == partitionId).findFirst().orElse(null); } - @Nullable - public PartitionChunk replaceChunkWith(PartitionChunk newChunk) - { - PartitionChunk oldChunk = null; - for (int i = 0; i < chunks.size(); i++) { - if (newChunk.getChunkNumber() == chunks.get(i).getChunkNumber()) { - oldChunk = chunks.set(i, newChunk); - break; - } - } - return oldChunk; - } - @Override public int getStartRootPartitionId() { 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 index 59ec3273b831..764b994f7e3c 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -224,9 +224,9 @@ private List>> findOvershadowedBy( /** * Handles addition of the atomicUpdateGroup to the given state */ - private void transitionStandbyGroupIfFull(AtomicUpdateGroup aug, State newStateOfAug) + private void transitionStandbyGroupIfFull(AtomicUpdateGroup aug, State stateOfAug) { - if (newStateOfAug == State.STANDBY) { + 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 @@ -253,17 +253,23 @@ private void addAtomicUpdateGroupWithState(AtomicUpdateGroup aug, State state transitionStandbyGroupIfFull(aug, state); } - public void addChunk(PartitionChunk chunk) + public boolean addChunk(PartitionChunk chunk) { // Sanity check. ExistingChunk should be usually null. final PartitionChunk existingChunk = knownPartitionChunks.put(chunk.getChunkNumber(), chunk); - if (existingChunk != null && !existingChunk.equals(chunk)) { - throw new ISE( - "existingChunk[%s] is different from newChunk[%s] for partitionId[%d]", - existingChunk, - chunk, - chunk.getChunkNumber() - ); + 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 @@ -281,22 +287,7 @@ public void addChunk(PartitionChunk chunk) atomicUpdateGroup = findAtomicUpdateGroupWith(chunk, State.VISIBLE); if (atomicUpdateGroup != null) { - // A new chunk of the same major version and partitionId can be added in segment handoff - // from stream ingestion tasks to historicals - final PartitionChunk existing = atomicUpdateGroup.replaceChunkWith(chunk); - if (existing == null) { - throw new ISE( - "Can't add a new partitionChunk[%s] to a visible atomicUpdateGroup[%s]", - chunk, - atomicUpdateGroup - ); - } else if (!chunk.equals(existing)) { - throw new ISE( - "WTH? a new partitionChunk[%s] has the same partitionId but different from existing chunk[%s]", - chunk, - existing - ); - } + atomicUpdateGroup.add(chunk); } else { final AtomicUpdateGroup newAtomicUpdateGroup = new AtomicUpdateGroup<>(chunk); @@ -315,6 +306,7 @@ public void addChunk(PartitionChunk chunk) } } } + return true; } /** 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 e08a34609015..26e34dacf76d 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 @@ -56,9 +56,9 @@ public PartitionHolder(PartitionHolder partitionHolder) this.overshadowableManager = new OvershadowableManager<>(partitionHolder.overshadowableManager); } - public void add(PartitionChunk chunk) + public boolean add(PartitionChunk chunk) { - overshadowableManager.addChunk(chunk); + return overshadowableManager.addChunk(chunk); } @Nullable 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 03d217cc08c0..6971833678f9 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -454,7 +454,7 @@ public void testOverlapSameVersionIsOkay() assertValues( Collections.singletonList( - createExpected("2011-01-01/2011-01-10", "2", 3) + createExpected("2011-01-01/2011-01-10", "2", 2) ), timeline.lookup(Intervals.of("2011-01-01/2011-01-10")) ); From dfe7985cc0e1f8d62a93c5f456e563bb03bf5001 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 18 Jul 2019 13:22:00 -0700 Subject: [PATCH 47/52] fix tc --- .../main/java/org/apache/druid/client/DataSourcesSnapshot.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java index 7b745beb61cd..dbd3ecb35124 100644 --- a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java +++ b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java @@ -146,7 +146,7 @@ public Iterable iterateAllUsedSegmentsInSnapshot() /** * This method builds timelines from all data sources and finds the overshadowed segments list * - * This method should be deduplicated with {@link VersionedIntervalTimeline#findOvershadowed()}: see + * This method should be deduplicated with {@link VersionedIntervalTimeline#findFullyOvershadowed()}: see * https://github.com/apache/incubator-druid/issues/8070. * * @return overshadowed segment Ids list From 920376b1d5e73b57731d770047d9a26e0f70361b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Jul 2019 17:26:31 -0700 Subject: [PATCH 48/52] remove remaining todos --- .../druid/indexing/common/task/AbstractBatchIndexTask.java | 1 - .../common/task/batch/parallel/ParallelIndexSupervisorTask.java | 2 -- 2 files changed, 3 deletions(-) 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 index 9f8ab3abac21..5a6ff19f9fe2 100644 --- 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 @@ -98,7 +98,6 @@ protected AbstractBatchIndexTask( * 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. */ - // TODO: remove this public abstract List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 17c0addc501e..817d18bbcff2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -406,8 +406,6 @@ public Response allocateSegment(DateTime timestamp, @Context final HttpServletRe } } - // TODO: add test for timeChunkLock without bucket intervals in granularitySpec - /** * Allocate a new segment for the given timestamp locally. * Since the segments returned by this method overwrites any existing segments, this method should be called only From b3b05db655414169625dc235ff9bf535c69a4ce1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Jul 2019 17:39:37 -0700 Subject: [PATCH 49/52] add comment for parallel index --- .../task/batch/parallel/ParallelIndexSubTask.java | 10 +++++++++- .../batch/parallel/ParallelIndexSupervisorTask.java | 7 +++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 88e0c15059f6..8d98df8358b5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -154,7 +154,15 @@ public String getType() @Override public boolean isReady(TaskActionClient taskActionClient) throws IOException { - if (!ingestionSchema.getDataSchema().getGranularitySpec().bucketIntervals().isPresent()) { + if (!ingestionSchema.getDataSchema().getGranularitySpec().bucketIntervals().isPresent() + && !ingestionSchema.getIOConfig().isAppendToExisting()) { + // If intervals are missing in the granularitySpec, parallel index task runs in "dynamic locking mode". + // In this mode, sub tasks ask new locks whenever they see a new row which is not covered by existing locks. + // If this task is overwriting existing segments, then we should know this task is changing segment granularity + // in advance to know what types of lock we should use. However, if intervals are missing, we can't know + // the segment granularity of existing segments until the task reads all data because we don't know what segments + // are going to be overwritten. As a result, we assume that segment granularity will be changed if intervals are + // missing force to use timeChunk lock. addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.getDataSchema().getGranularitySpec()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 817d18bbcff2..9a783dda48b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -220,6 +220,13 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception { if (!ingestionSchema.getDataSchema().getGranularitySpec().bucketIntervals().isPresent() && !ingestionSchema.getIOConfig().isAppendToExisting()) { + // If intervals are missing in the granularitySpec, parallel index task runs in "dynamic locking mode". + // In this mode, sub tasks ask new locks whenever they see a new row which is not covered by existing locks. + // If this task is overwriting existing segments, then we should know this task is changing segment granularity + // in advance to know what types of lock we should use. However, if intervals are missing, we can't know + // the segment granularity of existing segments until the task reads all data because we don't know what segments + // are going to be overwritten. As a result, we assume that segment granularity will be changed if intervals are + // missing force to use timeChunk lock. addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } return determineLockGranularityAndTryLock(taskActionClient, ingestionSchema.getDataSchema().getGranularitySpec()); From 610745ed3172447235637b463219c0d357eed28d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 23 Jul 2019 20:47:35 -0700 Subject: [PATCH 50/52] fix javadoc and typos --- .../common/task/AbstractBatchIndexTask.java | 34 +++++++++---------- .../task/CachingLocalSegmentAllocator.java | 2 +- 2 files changed, 18 insertions(+), 18 deletions(-) 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 index 5a6ff19f9fe2..c83980685e16 100644 --- 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 @@ -55,8 +55,8 @@ /** * Abstract class for batch tasks like {@link IndexTask}. - * Provides some methods ({@link #determineSegmentGranularity} and {@link #determineSegmentGranularity}) for easily acquiring task - * locks. + * Provides some methods such as {@link #determineSegmentGranularity}, {@link #findInputSegments}, + * and {@link #determineLockGranularityandTryLock} for easily acquiring task locks. */ public abstract class AbstractBatchIndexTask extends AbstractTask { @@ -155,7 +155,7 @@ boolean determineLockGranularityandTryLock(TaskActionClient client, List(segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet())) ); } else { - final LockGranularityDeterminResult result = determineSegmentGranularity(segments); + final LockGranularityDetermineResult result = determineSegmentGranularity(segments); useSegmentLock = result.lockGranularity == LockGranularity.SEGMENT; return tryLockWithDetermineResult(client, result); } } - private LockGranularityDeterminResult determineSegmentGranularity(TaskActionClient client, List intervals) + private LockGranularityDetermineResult determineSegmentGranularity(TaskActionClient client, List intervals) throws IOException { if (requireLockExistingSegments()) { if (isPerfectRollup()) { - log.info("Using timeChunk lock for perfrect rollup"); - return new LockGranularityDeterminResult(LockGranularity.TIME_CHUNK, intervals, null); + 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(), @@ -202,15 +202,15 @@ private LockGranularityDeterminResult determineSegmentGranularity(TaskActionClie return determineSegmentGranularity(findSegmentsToLock(client, intervals)); } else { log.info("Using segment lock for empty intervals"); - return new LockGranularityDeterminResult(LockGranularity.SEGMENT, null, Collections.emptyList()); + 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 LockGranularityDeterminResult(LockGranularity.SEGMENT, null, Collections.emptyList()); + return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, Collections.emptyList()); } } - private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranularityDeterminResult result) + private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranularityDetermineResult result) throws IOException { if (result.lockGranularity == LockGranularity.TIME_CHUNK) { @@ -245,13 +245,13 @@ private boolean tryTimeChunkLock(TaskActionClient client, List interva return true; } - private LockGranularityDeterminResult determineSegmentGranularity(List segments) + private LockGranularityDetermineResult determineSegmentGranularity(List 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 LockGranularityDeterminResult(LockGranularity.SEGMENT, null, Collections.emptyList()); + return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, Collections.emptyList()); } if (requireLockExistingSegments()) { @@ -269,7 +269,7 @@ private LockGranularityDeterminResult determineSegmentGranularity(List findInputSegments( } } - private static class LockGranularityDeterminResult + private static class LockGranularityDetermineResult { private final LockGranularity lockGranularity; @Nullable @@ -370,7 +370,7 @@ private static class LockGranularityDeterminResult @Nullable private final List segments; // null for timeChunkLock - private LockGranularityDeterminResult( + private LockGranularityDetermineResult( LockGranularity lockGranularity, @Nullable List intervals, @Nullable List segments 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 index 8803f9db041c..8d5680e6ed11 100644 --- 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 @@ -80,7 +80,7 @@ class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator for (SegmentIdWithShardSpec segmentIdentifier : idsPerInterval) { shardSpecMap.computeIfAbsent(interval, k -> new ArrayList<>()).add(segmentIdentifier.getShardSpec()); - // The shardSpecs for partitinoing and publishing can be different if isExtendableShardSpecs = true. + // The shardSpecs for partitioning and publishing can be different if isExtendableShardSpecs = true. sequenceNameToSegmentId.put(getSequenceName(interval, segmentIdentifier.getShardSpec()), segmentIdentifier); } } From cb2de9a5d77dd3771bd510377b40efa9d543233e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 24 Jul 2019 12:46:00 -0700 Subject: [PATCH 51/52] typo --- .../common/task/batch/parallel/ParallelIndexSupervisorTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 9a783dda48b3..8765884d5e8c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -440,7 +440,7 @@ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException Interval interval; String version; if (bucketIntervals.isPresent()) { - // If granularity spec has explicit intervals, we just need to find the version accociated to the interval. + // If granularity spec has explicit intervals, we just need to find the version associated to the interval. // This is because we should have gotten all required locks up front when the task starts up. final Optional maybeInterval = granularitySpec.bucketInterval(timestamp); if (!maybeInterval.isPresent()) { From c1d1bf7203c12b865028b9e21a0944eeaf725c48 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 24 Jul 2019 14:32:14 -0700 Subject: [PATCH 52/52] address comments --- .../apache/druid/timeline/DataSegment.java | 2 +- .../apache/druid/timeline/Overshadowable.java | 4 +-- .../timeline/VersionedIntervalTimeline.java | 4 +-- .../partition/OvershadowableManager.java | 30 +++++++++++++++---- .../common/task/AbstractBatchIndexTask.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 4 ++- .../indexer/AbstractITBatchIndexTest.java | 2 +- .../segment/ReferenceCountingSegment.java | 2 +- .../druid/segment/SchemalessIndexTest.java | 2 +- .../apache/druid/client/SegmentLoadInfo.java | 4 +-- .../druid/client/selector/ServerSelector.java | 4 +-- .../druid/segment/realtime/plumber/Sink.java | 2 +- .../client/CachingClusteredClientTest.java | 2 +- .../server/shard/NumberedShardSpecTest.java | 2 +- 14 files changed, 43 insertions(+), 23 deletions(-) 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 a8e2fe77f42a..c427f6a04ea6 100644 --- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -276,7 +276,7 @@ public SegmentId getId() } @Override - public boolean isOvershadow(DataSegment other) + 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()); diff --git a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java index b051229d1681..69b4336f459a 100644 --- a/core/src/main/java/org/apache/druid/timeline/Overshadowable.java +++ b/core/src/main/java/org/apache/druid/timeline/Overshadowable.java @@ -30,9 +30,9 @@ public interface Overshadowable { /** - * Returns true if this segment overshadows the other segment. + * Returns true if this overshadowable overshadows the other. */ - default boolean isOvershadow(T other) + default boolean overshadows(T other) { final int majorVersionCompare = getVersion().compareTo(other.getVersion()); if (majorVersionCompare == 0) { 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 b8704f6eb32b..4998c41c0f65 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -419,7 +419,7 @@ public boolean isOvershadowed(Interval interval, VersionType version, ObjectType final int majorVersionCompare = versionComparator.compare(version, entry.getVersion()); if (majorVersionCompare == 0) { for (PartitionChunk chunk : entry.partitionHolder) { - if (chunk.getObject().isOvershadow(object)) { + if (chunk.getObject().overshadows(object)) { return true; } } @@ -454,7 +454,7 @@ public boolean isOvershadowed(Interval interval, VersionType version, ObjectType if (versionCompare > 0) { return false; } else if (versionCompare == 0) { - if (timelineEntry.partitionHolder.stream().noneMatch(chunk -> chunk.getObject().isOvershadow(object))) { + if (timelineEntry.partitionHolder.stream().noneMatch(chunk -> chunk.getObject().overshadows(object))) { return false; } } 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 index 764b994f7e3c..97e8b9aae92f 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java @@ -188,14 +188,22 @@ private List>> findOvershadowedBy( 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>> findOvershadowedBy( RootPartitionRange rangeOfAug, short minorVersion, State fromState ) { - Entry>> current = getStateMap(fromState) - .floorEntry(rangeOfAug); + final TreeMap>> stateMap = getStateMap(fromState); + Entry>> current = stateMap.floorEntry(rangeOfAug); if (current == null) { return Collections.emptyList(); @@ -203,8 +211,9 @@ private List>> findOvershadowedBy( // Find the first key for searching for overshadowed atomicUpdateGroup while (true) { - final Entry>> lowerEntry = getStateMap(fromState) - .lowerEntry(current.getKey()); + final Entry>> lowerEntry = stateMap.lowerEntry( + current.getKey() + ); if (lowerEntry != null && lowerEntry.getKey().startPartitionId == rangeOfAug.startPartitionId) { current = lowerEntry; } else { @@ -212,11 +221,20 @@ private List>> findOvershadowedBy( } } + // 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>> 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> 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 = getStateMap(fromState).higherEntry(current.getKey()); + current = stateMap.higherEntry(current.getKey()); } return found; } @@ -296,7 +314,7 @@ public boolean addChunk(PartitionChunk chunk) .values() .stream() .flatMap(map -> map.values().stream()) - .anyMatch(group -> group.isOvershadow(newAtomicUpdateGroup)); + .anyMatch(group -> group.overshadows(newAtomicUpdateGroup)); if (overshadowed) { addAtomicUpdateGroupWithState(newAtomicUpdateGroup, State.OVERSHADOWED); 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 index c83980685e16..b00c646385a3 100644 --- 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 @@ -225,7 +225,7 @@ private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranular private boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException { - // In this case, the intervals to lock must be alighed with segmentGranularity if it's defined + // In this case, the intervals to lock must be aligned with segmentGranularity if it's defined final Set uniqueIntervals = new HashSet<>(); for (Interval interval : JodaUtils.condenseIntervals(intervals)) { final Granularity segmentGranularity = getSegmentGranularity(); 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 1c74dbc66074..b6a4126f8fac 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 @@ -938,10 +938,12 @@ private TaskStatus generateAndPublishSegments( final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); log.info("Pushed segments[%s]", pushed.getSegments()); - // Probably we can publish atomicUpdateGroup along with segments. + // If we use timeChunk lock, then we don't have to specify what segments will be overwritten because + // it will just overwrite all segments overlapped with the new segments. final Set inputSegments = isUseSegmentLock() ? getSegmentLockHelper().getLockedExistingSegments() : null; + // Probably we can publish atomicUpdateGroup along with segments. final SegmentsAndMetadata published = awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout); ingestionState = IngestionState.COMPLETED; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index cb3891a1f0af..dd2945887b5e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -206,7 +206,7 @@ private void submitTaskAndWait(String taskSpec, String dataSourceName, boolean w .stream() .flatMap(holder -> holder.getObject().stream()) .anyMatch(chunk -> oldVersions.stream() - .anyMatch(oldSegment -> chunk.getObject().isOvershadow(oldSegment))); + .anyMatch(oldSegment -> chunk.getObject().overshadows(oldSegment))); }, "See a new version" ); diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java index da834495c593..335cd1ea8c19 100644 --- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java @@ -179,7 +179,7 @@ public T as(Class clazz) } @Override - public boolean isOvershadow(ReferenceCountingSegment other) + public boolean overshadows(ReferenceCountingSegment other) { if (baseSegment.getId().getDataSource().equals(other.baseSegment.getId().getDataSource()) && baseSegment.getId().getInterval().overlaps(other.baseSegment.getId().getInterval())) { diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java index 039a94f17134..43a9ba59c2a5 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java @@ -583,7 +583,7 @@ private static class OvershadowableFile implements Overshadowable